diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 01b107b3ea7595..a8cbac9326dd75 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -1640,11 +1640,13 @@ void drop_tablet_callback(StorageEngine& engine, const TAgentTaskRequest& req) { dropped_tablet->tablet_uid()); LOG_INFO("successfully drop tablet") .tag("signature", req.signature) - .tag("tablet_id", drop_tablet_req.tablet_id); + .tag("tablet_id", drop_tablet_req.tablet_id) + .tag("replica_id", drop_tablet_req.replica_id); } else { LOG_WARNING("failed to drop tablet") .tag("signature", req.signature) .tag("tablet_id", drop_tablet_req.tablet_id) + .tag("replica_id", drop_tablet_req.replica_id) .error(status); } diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 990588ec73993d..6e610c395e8b73 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -33,6 +33,7 @@ #include "util/uuid_generator.h" namespace doris { +#include "common/compile_check_begin.h" using namespace ErrorCode; bvar::Adder cumu_output_size("cumu_compaction", "output_size"); @@ -490,8 +491,10 @@ Status CloudCumulativeCompaction::pick_rowsets_to_compact() { } int64_t max_score = config::cumulative_compaction_max_deltas; - auto process_memory_usage = doris::GlobalMemoryArbitrator::process_memory_usage(); - bool memory_usage_high = process_memory_usage > MemInfo::soft_mem_limit() * 0.8; + double process_memory_usage = + static_cast(doris::GlobalMemoryArbitrator::process_memory_usage()); + bool memory_usage_high = + process_memory_usage > static_cast(MemInfo::soft_mem_limit()) * 0.8; if (cloud_tablet()->last_compaction_status.is() || memory_usage_high) { max_score = std::max(config::cumulative_compaction_max_deltas / @@ -621,4 +624,5 @@ void CloudCumulativeCompaction::do_lease() { } } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction.h b/be/src/cloud/cloud_cumulative_compaction.h index 1159dcb59ceef1..87fc0b62c9c389 100644 --- a/be/src/cloud/cloud_cumulative_compaction.h +++ b/be/src/cloud/cloud_cumulative_compaction.h @@ -24,6 +24,7 @@ #include "olap/compaction.h" namespace doris { +#include "common/compile_check_begin.h" class CloudCumulativeCompaction : public CloudCompactionMixin { public: @@ -60,4 +61,5 @@ class CloudCumulativeCompaction : public CloudCompactionMixin { Version _last_delete_version {-1, -1}; }; +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction_policy.cpp b/be/src/cloud/cloud_cumulative_compaction_policy.cpp index 5a9879387b2327..6445b47fc5963f 100644 --- a/be/src/cloud/cloud_cumulative_compaction_policy.cpp +++ b/be/src/cloud/cloud_cumulative_compaction_policy.cpp @@ -31,6 +31,7 @@ #include "olap/tablet_meta.h" namespace doris { +#include "common/compile_check_begin.h" CloudSizeBasedCumulativeCompactionPolicy::CloudSizeBasedCumulativeCompactionPolicy( int64_t promotion_size, double promotion_ratio, int64_t promotion_min_size, @@ -48,7 +49,7 @@ int64_t CloudSizeBasedCumulativeCompactionPolicy::_level_size(const int64_t size return (int64_t)1 << (sizeof(size) * 8 - 1 - __builtin_clzl(size)); } -int32_t CloudSizeBasedCumulativeCompactionPolicy::pick_input_rowsets( +int64_t CloudSizeBasedCumulativeCompactionPolicy::pick_input_rowsets( CloudTablet* tablet, const std::vector& candidate_rowsets, const int64_t max_compaction_score, const int64_t min_compaction_score, std::vector* input_rowsets, Version* last_delete_version, @@ -114,8 +115,8 @@ int32_t CloudSizeBasedCumulativeCompactionPolicy::pick_input_rowsets( size_t new_compaction_score = *compaction_score; while (rs_begin != input_rowsets->end()) { auto& rs_meta = (*rs_begin)->rowset_meta(); - int current_level = _level_size(rs_meta->total_disk_size()); - int remain_level = _level_size(total_size - rs_meta->total_disk_size()); + int64_t current_level = _level_size(rs_meta->total_disk_size()); + int64_t remain_level = _level_size(total_size - rs_meta->total_disk_size()); // if current level less then remain level, input rowsets contain current rowset // and process return; otherwise, input rowsets do not contain current rowset. if (current_level <= remain_level) { @@ -185,7 +186,7 @@ int32_t CloudSizeBasedCumulativeCompactionPolicy::pick_input_rowsets( } int64_t CloudSizeBasedCumulativeCompactionPolicy::cloud_promotion_size(CloudTablet* t) const { - int64_t promotion_size = int64_t(t->base_size() * _promotion_ratio); + int64_t promotion_size = int64_t(static_cast(t->base_size()) * _promotion_ratio); // promotion_size is between _size_based_promotion_size and _size_based_promotion_min_size return promotion_size > _promotion_size ? _promotion_size : promotion_size < _promotion_min_size ? _promotion_min_size @@ -215,7 +216,7 @@ int64_t CloudSizeBasedCumulativeCompactionPolicy::new_cumulative_point( : last_cumulative_point; } -int32_t CloudTimeSeriesCumulativeCompactionPolicy::pick_input_rowsets( +int64_t CloudTimeSeriesCumulativeCompactionPolicy::pick_input_rowsets( CloudTablet* tablet, const std::vector& candidate_rowsets, const int64_t max_compaction_score, const int64_t min_compaction_score, std::vector* input_rowsets, Version* last_delete_version, @@ -377,4 +378,5 @@ int64_t CloudTimeSeriesCumulativeCompactionPolicy::new_cumulative_point( return output_rowset->end_version() + 1; } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_cumulative_compaction_policy.h b/be/src/cloud/cloud_cumulative_compaction_policy.h index c142a8a6d3dffe..9373728547241b 100644 --- a/be/src/cloud/cloud_cumulative_compaction_policy.h +++ b/be/src/cloud/cloud_cumulative_compaction_policy.h @@ -30,6 +30,7 @@ #include "olap/rowset/rowset_meta.h" namespace doris { +#include "common/compile_check_begin.h" class Tablet; struct Version; @@ -44,7 +45,7 @@ class CloudCumulativeCompactionPolicy { virtual int64_t new_compaction_level(const std::vector& input_rowsets) = 0; - virtual int32_t pick_input_rowsets(CloudTablet* tablet, + virtual int64_t pick_input_rowsets(CloudTablet* tablet, const std::vector& candidate_rowsets, const int64_t max_compaction_score, const int64_t min_compaction_score, @@ -71,7 +72,7 @@ class CloudSizeBasedCumulativeCompactionPolicy : public CloudCumulativeCompactio return 0; } - int32_t pick_input_rowsets(CloudTablet* tablet, + int64_t pick_input_rowsets(CloudTablet* tablet, const std::vector& candidate_rowsets, const int64_t max_compaction_score, const int64_t min_compaction_score, @@ -106,7 +107,7 @@ class CloudTimeSeriesCumulativeCompactionPolicy : public CloudCumulativeCompacti int64_t new_compaction_level(const std::vector& input_rowsets) override; - int32_t pick_input_rowsets(CloudTablet* tablet, + int64_t pick_input_rowsets(CloudTablet* tablet, const std::vector& candidate_rowsets, const int64_t max_compaction_score, const int64_t min_compaction_score, @@ -115,4 +116,5 @@ class CloudTimeSeriesCumulativeCompactionPolicy : public CloudCumulativeCompacti bool allow_delete = false) override; }; +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_delete_bitmap_action.cpp b/be/src/cloud/cloud_delete_bitmap_action.cpp index 86cc535e1bc88e..3d834bfe7b373c 100644 --- a/be/src/cloud/cloud_delete_bitmap_action.cpp +++ b/be/src/cloud/cloud_delete_bitmap_action.cpp @@ -50,6 +50,7 @@ #include "util/stopwatch.hpp" namespace doris { +#include "common/compile_check_begin.h" using namespace ErrorCode; namespace { @@ -177,4 +178,5 @@ void CloudDeleteBitmapAction::handle(HttpRequest* req) { } } +#include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/cloud/cloud_delete_bitmap_action.h b/be/src/cloud/cloud_delete_bitmap_action.h index 35739a7373efc8..ce507ee9991757 100644 --- a/be/src/cloud/cloud_delete_bitmap_action.h +++ b/be/src/cloud/cloud_delete_bitmap_action.h @@ -27,6 +27,7 @@ #include "olap/tablet.h" namespace doris { +#include "common/compile_check_begin.h" class HttpRequest; class ExecEnv; @@ -52,4 +53,5 @@ class CloudDeleteBitmapAction : public HttpHandlerWithAuth { CloudStorageEngine& _engine; DeleteBitmapActionType _delete_bitmap_action_type; }; +#include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp index 6abc3958650ef6..83f19881b72bca 100644 --- a/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp +++ b/be/src/cloud/cloud_engine_calc_delete_bitmap_task.cpp @@ -34,6 +34,7 @@ #include "runtime/memory/mem_tracker_limiter.h" namespace doris { +#include "common/compile_check_begin.h" CloudEngineCalcDeleteBitmapTask::CloudEngineCalcDeleteBitmapTask( CloudStorageEngine& engine, const TCalcDeleteBitmapRequest& cal_delete_bitmap_req, @@ -218,7 +219,7 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { .base_compaction_cnt = _ms_base_compaction_cnt, .cumulative_compaction_cnt = _ms_cumulative_compaction_cnt, .cumulative_point = _ms_cumulative_point}; - auto update_delete_bitmap_time_us = 0; + int64_t update_delete_bitmap_time_us = 0; if (txn_info.publish_status && (*(txn_info.publish_status) == PublishStatus::SUCCEED) && _version == previous_publish_info.publish_version && _ms_base_compaction_cnt == previous_publish_info.base_compaction_cnt && @@ -253,4 +254,5 @@ Status CloudTabletCalcDeleteBitmapTask::handle() const { return status; } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index bf32450b886751..6659164d4f78f1 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -64,6 +64,7 @@ #include "util/thrift_rpc_helper.h" namespace doris::cloud { +#include "common/compile_check_begin.h" using namespace ErrorCode; Status bthread_fork_join(const std::vector>& tasks, int concurrency) { @@ -717,7 +718,7 @@ Status CloudMetaMgr::sync_tablet_delete_bitmap(CloudTablet* tablet, int64_t old_ "rowset_ids.size={},segment_ids.size={},vers.size={},delete_bitmaps.size={}", rowset_ids.size(), segment_ids.size(), vers.size(), delete_bitmaps.size()); } - for (size_t i = 0; i < rowset_ids.size(); i++) { + for (int i = 0; i < rowset_ids.size(); i++) { RowsetId rst_id; rst_id.init(rowset_ids[i]); delete_bitmap->merge( @@ -757,10 +758,10 @@ Status CloudMetaMgr::prepare_rowset(const RowsetMeta& rs_meta, Status st = retry_rpc("prepare rowset", req, &resp, &MetaService_Stub::prepare_rowset); if (!st.ok() && resp.status().code() == MetaServiceCode::ALREADY_EXISTED) { if (existed_rs_meta != nullptr && resp.has_existed_rowset_meta()) { - RowsetMetaPB doris_rs_meta = + RowsetMetaPB doris_rs_meta_tmp = cloud_rowset_meta_to_doris(std::move(*resp.mutable_existed_rowset_meta())); *existed_rs_meta = std::make_shared(); - (*existed_rs_meta)->init_from_pb(doris_rs_meta); + (*existed_rs_meta)->init_from_pb(doris_rs_meta_tmp); } return Status::AlreadyExist("failed to prepare rowset: {}", resp.status().msg()); } @@ -1286,4 +1287,5 @@ int64_t CloudMetaMgr::get_inverted_index_file_szie(const RowsetMeta& rs_meta) { return total_inverted_index_size; } +#include "common/compile_check_end.h" } // namespace doris::cloud diff --git a/be/src/cloud/cloud_meta_mgr.h b/be/src/cloud/cloud_meta_mgr.h index c49b036ad90c15..913ef59489a1b3 100644 --- a/be/src/cloud/cloud_meta_mgr.h +++ b/be/src/cloud/cloud_meta_mgr.h @@ -27,6 +27,7 @@ #include "util/s3_util.h" namespace doris { +#include "common/compile_check_begin.h" class DeleteBitmap; class StreamLoadContext; @@ -124,4 +125,5 @@ class CloudMetaMgr { }; } // namespace cloud +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_storage_engine.cpp b/be/src/cloud/cloud_storage_engine.cpp index 5d7b445917aa20..a49809bd59c02b 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -52,6 +52,7 @@ #include "util/parse_util.h" namespace doris { +#include "common/compile_check_begin.h" using namespace std::literals; @@ -166,7 +167,8 @@ Status CloudStorageEngine::open() { _memtable_flush_executor = std::make_unique(); // Use file cache disks number - _memtable_flush_executor->init(io::FileCacheFactory::instance()->get_cache_instance_size()); + _memtable_flush_executor->init( + static_cast(io::FileCacheFactory::instance()->get_cache_instance_size())); _calc_delete_bitmap_executor = std::make_unique(); _calc_delete_bitmap_executor->init(); @@ -308,7 +310,7 @@ void CloudStorageEngine::_check_file_cache_ttl_block_valid() { for (const auto& rowset : rowsets) { int64_t ttl_seconds = tablet->tablet_meta()->ttl_seconds(); if (rowset->newest_write_timestamp() + ttl_seconds <= UnixSeconds()) continue; - for (int64_t seg_id = 0; seg_id < rowset->num_segments(); seg_id++) { + for (uint32_t seg_id = 0; seg_id < rowset->num_segments(); seg_id++) { auto hash = Segment::file_cache_key(rowset->rowset_id().to_string(), seg_id); auto* file_cache = io::FileCacheFactory::instance()->get_by_path(hash); file_cache->update_ttl_atime(hash); @@ -337,11 +339,11 @@ void CloudStorageEngine::sync_storage_vault() { for (auto& [id, vault_info, path_format] : vault_infos) { auto fs = get_filesystem(id); - auto st = (fs == nullptr) - ? std::visit(VaultCreateFSVisitor {id, path_format}, vault_info) - : std::visit(RefreshFSVaultVisitor {id, std::move(fs), path_format}, - vault_info); - if (!st.ok()) [[unlikely]] { + auto status = (fs == nullptr) + ? std::visit(VaultCreateFSVisitor {id, path_format}, vault_info) + : std::visit(RefreshFSVaultVisitor {id, std::move(fs), path_format}, + vault_info); + if (!status.ok()) [[unlikely]] { LOG(WARNING) << vault_process_error(id, vault_info, std::move(st)); } } @@ -491,13 +493,13 @@ void CloudStorageEngine::_compaction_tasks_producer_callback() { /// If it is not cleaned up, the reference count of the tablet will always be greater than 1, /// thus cannot be collected by the garbage collector. (TabletManager::start_trash_sweep) for (const auto& tablet : tablets_compaction) { - Status st = submit_compaction_task(tablet, compaction_type); - if (st.ok()) continue; - if ((!st.is() && - !st.is()) || + Status status = submit_compaction_task(tablet, compaction_type); + if (status.ok()) continue; + if ((!status.is() && + !status.is()) || VLOG_DEBUG_IS_ON) { LOG(WARNING) << "failed to submit compaction task for tablet: " - << tablet->tablet_id() << ", err: " << st; + << tablet->tablet_id() << ", err: " << status; } } interval = config::generate_compaction_tasks_interval_ms; @@ -531,7 +533,8 @@ std::vector CloudStorageEngine::_generate_cloud_compaction_task int num_cumu = std::accumulate(submitted_cumu_compactions.begin(), submitted_cumu_compactions.end(), 0, [](int a, auto& b) { return a + b.second.size(); }); - int num_base = submitted_base_compactions.size() + submitted_full_compactions.size(); + int num_base = + static_cast(submitted_base_compactions.size() + submitted_full_compactions.size()); int n = thread_per_disk - num_cumu - num_base; if (compaction_type == CompactionType::BASE_COMPACTION) { // We need to reserve at least one thread for cumulative compaction, @@ -809,7 +812,7 @@ Status CloudStorageEngine::get_compaction_status_json(std::string* result) { // cumu std::string_view cumu = "CumulativeCompaction"; rapidjson::Value cumu_key; - cumu_key.SetString(cumu.data(), cumu.length(), root.GetAllocator()); + cumu_key.SetString(cumu.data(), static_cast(cumu.length()), root.GetAllocator()); rapidjson::Document cumu_arr; cumu_arr.SetArray(); for (auto& [tablet_id, v] : _submitted_cumu_compactions) { @@ -821,7 +824,7 @@ Status CloudStorageEngine::get_compaction_status_json(std::string* result) { // base std::string_view base = "BaseCompaction"; rapidjson::Value base_key; - base_key.SetString(base.data(), base.length(), root.GetAllocator()); + base_key.SetString(base.data(), static_cast(base.length()), root.GetAllocator()); rapidjson::Document base_arr; base_arr.SetArray(); for (auto& [tablet_id, _] : _submitted_base_compactions) { @@ -844,4 +847,5 @@ std::shared_ptr CloudStorageEngine::cumu_compac return _cumulative_compaction_policies.at(compaction_policy); } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index ebe06ac558fb6e..fe9494ef149bec 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -50,6 +50,7 @@ #include "vec/common/schema_util.h" namespace doris { +#include "common/compile_check_begin.h" using namespace ErrorCode; static constexpr int COMPACTION_DELETE_BITMAP_LOCK_ID = -1; @@ -219,6 +220,7 @@ Status CloudTablet::sync_if_not_running() { } TabletSchemaSPtr CloudTablet::merged_tablet_schema() const { + std::shared_lock rlock(_meta_lock); return _merged_tablet_schema; } @@ -380,7 +382,7 @@ void CloudTablet::delete_rowsets(const std::vector& to_delete, _tablet_meta->modify_rs_metas({}, rs_metas, false); } -int CloudTablet::delete_expired_stale_rowsets() { +uint64_t CloudTablet::delete_expired_stale_rowsets() { std::vector expired_rowsets; int64_t expired_stale_sweep_endtime = ::time(nullptr) - config::tablet_rowset_stale_sweep_time_sec; @@ -397,8 +399,8 @@ int CloudTablet::delete_expired_stale_rowsets() { } for (int64_t path_id : path_ids) { - int start_version = -1; - int end_version = -1; + int64_t start_version = -1; + int64_t end_version = -1; // delete stale versions in version graph auto version_path = _timestamped_version_tracker.fetch_and_delete_path_by_id(path_id); for (auto& v_ts : version_path->timestamped_versions()) { @@ -539,7 +541,7 @@ Result> CloudTablet::create_transient_rowset_write return RowsetFactory::create_rowset_writer(_engine, context, false) .transform([&](auto&& writer) { - writer->set_segment_start_id(rowset.num_segments()); + writer->set_segment_start_id(static_cast(rowset.num_segments())); return writer; }); } @@ -617,7 +619,8 @@ void CloudTablet::get_compaction_status(std::string* json_result) { } rapidjson::Value value; std::string version_str = rowset->get_rowset_info_str(); - value.SetString(version_str.c_str(), version_str.length(), versions_arr.GetAllocator()); + value.SetString(version_str.c_str(), static_cast(version_str.length()), + versions_arr.GetAllocator()); versions_arr.PushBack(value, versions_arr.GetAllocator()); last_version = ver.second; } @@ -630,7 +633,7 @@ void CloudTablet::get_compaction_status(std::string* json_result) { for (auto& rowset : stale_rowsets) { rapidjson::Value value; std::string version_str = rowset->get_rowset_info_str(); - value.SetString(version_str.c_str(), version_str.length(), + value.SetString(version_str.c_str(), static_cast(version_str.length()), stale_versions_arr.GetAllocator()); stale_versions_arr.PushBack(value, stale_versions_arr.GetAllocator()); } @@ -919,4 +922,5 @@ void CloudTablet::build_tablet_report_info(TTabletInfo* tablet_info) { // but it may be used in the future. } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 3433c357322c1e..458964bac2cdda 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -92,7 +92,7 @@ class CloudTablet final : public BaseTablet { void clear_cache() override; // Return number of deleted stale rowsets - int delete_expired_stale_rowsets(); + uint64_t delete_expired_stale_rowsets(); bool has_stale_rowsets() const { return !_stale_rs_version_map.empty(); } diff --git a/be/src/cloud/config.cpp b/be/src/cloud/config.cpp index 2d7be6b4971121..feb81d0a074b04 100644 --- a/be/src/cloud/config.cpp +++ b/be/src/cloud/config.cpp @@ -20,6 +20,7 @@ #include "common/status.h" namespace doris::config { +#include "common/compile_check_begin.h" DEFINE_String(deploy_mode, ""); DEFINE_mString(cloud_unique_id, ""); @@ -76,4 +77,5 @@ DEFINE_mInt32(tablet_txn_info_min_expired_seconds, "120"); DEFINE_mBool(enable_use_cloud_unique_id_from_fe, "true"); DEFINE_mBool(enable_cloud_tablet_report, "false"); +#include "common/compile_check_end.h" } // namespace doris::config diff --git a/be/src/cloud/config.h b/be/src/cloud/config.h index b345e6355921a4..a8a7c0c48ec91f 100644 --- a/be/src/cloud/config.h +++ b/be/src/cloud/config.h @@ -20,6 +20,7 @@ #include "common/config.h" namespace doris::config { +#include "common/compile_check_begin.h" DECLARE_String(deploy_mode); // deprecated do not configure directly @@ -110,4 +111,5 @@ DECLARE_mBool(enable_use_cloud_unique_id_from_fe); DECLARE_Bool(enable_cloud_tablet_report); +#include "common/compile_check_end.h" } // namespace doris::config diff --git a/be/src/common/cgroup_memory_ctl.cpp b/be/src/common/cgroup_memory_ctl.cpp index a29432bdb4ede5..dddcbd50338d82 100644 --- a/be/src/common/cgroup_memory_ctl.cpp +++ b/be/src/common/cgroup_memory_ctl.cpp @@ -27,6 +27,7 @@ #include "common/status.h" #include "util/cgroup_util.h" +#include "util/error_util.h" namespace doris { @@ -84,14 +85,23 @@ struct CgroupsV2Reader : CGroupMemoryCtl::ICgroupsReader { : _mount_file_dir(std::move(mount_file_dir)) {} Status read_memory_limit(int64_t* value) override { - RETURN_IF_ERROR(CGroupUtil::read_int_line_from_cgroup_file((_mount_file_dir / "memory.max"), - value)); + std::filesystem::path file_path = _mount_file_dir / "memory.max"; + std::string line; + std::ifstream file_stream(file_path, std::ios::in); + getline(file_stream, line); + if (file_stream.fail() || file_stream.bad()) { + return Status::CgroupError("Error reading {}: {}", file_path.string(), + get_str_err_msg()); + } + if (line == "max") { + *value = std::numeric_limits::max(); + return Status::OK(); + } + RETURN_IF_ERROR(CGroupUtil::read_int_line_from_cgroup_file(file_path, value)); return Status::OK(); } Status read_memory_usage(int64_t* value) override { - // memory.current contains a single number - // the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 RETURN_IF_ERROR(CGroupUtil::read_int_line_from_cgroup_file( (_mount_file_dir / "memory.current"), value)); std::unordered_map metrics_map; @@ -100,7 +110,12 @@ struct CgroupsV2Reader : CGroupMemoryCtl::ICgroupsReader { if (*value < metrics_map["inactive_file"]) { return Status::CgroupError("CgroupsV2Reader read_memory_usage negative memory usage"); } + // the reason why we subtract inactive_file described here: + // https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 *value -= metrics_map["inactive_file"]; + // Part of "slab" that might be reclaimed, such as dentries and inodes. + // https://arthurchiao.art/blog/cgroupv2-zh/ + *value -= metrics_map["slab_reclaimable"]; return Status::OK(); } diff --git a/be/src/common/compile_check_begin.h b/be/src/common/compile_check_begin.h new file mode 100644 index 00000000000000..4d860d39d1cf72 --- /dev/null +++ b/be/src/common/compile_check_begin.h @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#ifdef COMPILE_CHECK +#error The handling of compile_check_begin.h and compile_check_end.h is not done correctly. +#endif + +#define COMPILE_CHECK +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic error "-Wconversion" +#pragma clang diagnostic error "-Wshadow" +#pragma clang diagnostic ignored "-Wsign-conversion" +#pragma clang diagnostic ignored "-Wfloat-conversion" +#endif + +//#include "common/compile_check_begin.h" diff --git a/be/src/common/compile_check_end.h b/be/src/common/compile_check_end.h new file mode 100644 index 00000000000000..40df41b6bdfc6c --- /dev/null +++ b/be/src/common/compile_check_end.h @@ -0,0 +1,23 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#ifdef __clang__ +#pragma clang diagnostic pop +#endif +#undef COMPILE_CHECK + +// #include "common/compile_check_end.h" diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index a0949c32684544..b73c4aae7a0f1c 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -152,7 +152,10 @@ DEFINE_mInt64(stacktrace_in_alloc_large_memory_bytes, "2147483648"); DEFINE_mInt64(crash_in_alloc_large_memory_bytes, "-1"); -// If memory tracker value is inaccurate, BE will crash. usually used in test environments, default value is false. +// The actual meaning of this parameter is `debug_memory`. +// 1. crash in memory tracker inaccurate, if memory tracker value is inaccurate, BE will crash. +// usually used in test environments, default value is false. +// 2. print more memory logs. DEFINE_mBool(crash_in_memory_tracker_inaccurate, "false"); // default is true. if any memory tracking in Orphan mem tracker will report error. @@ -229,6 +232,8 @@ DEFINE_mInt32(max_download_speed_kbps, "50000"); DEFINE_mInt32(download_low_speed_limit_kbps, "50"); // download low speed time(seconds) DEFINE_mInt32(download_low_speed_time, "300"); +// whether to download small files in batch +DEFINE_mBool(enable_batch_download, "false"); DEFINE_String(sys_log_dir, ""); DEFINE_String(user_function_dir, "${DORIS_HOME}/lib/udf"); @@ -413,6 +418,8 @@ DEFINE_mDouble(base_compaction_min_data_ratio, "0.3"); DEFINE_mInt64(base_compaction_dup_key_max_file_size_mbytes, "1024"); DEFINE_Bool(enable_skip_tablet_compaction, "true"); +DEFINE_mInt32(skip_tablet_compaction_second, "10"); + // output rowset of cumulative compaction total disk size exceed this config size, // this rowset will be given to base compaction, unit is m byte. DEFINE_mInt64(compaction_promotion_size_mbytes, "1024"); @@ -445,10 +452,10 @@ DEFINE_mInt32(cumulative_compaction_max_deltas_factor, "10"); DEFINE_mInt32(multi_get_max_threads, "10"); // The upper limit of "permits" held by all compaction tasks. This config can be set to limit memory consumption for compaction. -DEFINE_mInt64(total_permits_for_compaction_score, "10000"); +DEFINE_mInt64(total_permits_for_compaction_score, "1000000"); // sleep interval in ms after generated compaction tasks -DEFINE_mInt32(generate_compaction_tasks_interval_ms, "10"); +DEFINE_mInt32(generate_compaction_tasks_interval_ms, "100"); // sleep interval in second after update replica infos DEFINE_mInt32(update_replica_infos_interval_seconds, "60"); @@ -1389,6 +1396,7 @@ DEFINE_mBool(enable_delete_bitmap_merge_on_compaction, "false"); // Enable validation to check the correctness of table size. DEFINE_Bool(enable_table_size_correctness_check, "false"); +DEFINE_Bool(force_regenerate_rowsetid_on_start_error, "false"); // clang-format off #ifdef BE_TEST diff --git a/be/src/common/config.h b/be/src/common/config.h index 86a6b11a2ae7cc..95b04b56a5c6c7 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -201,7 +201,10 @@ DECLARE_mInt64(stacktrace_in_alloc_large_memory_bytes); // modify this parameter to crash when large memory allocation occur will help DECLARE_mInt64(crash_in_alloc_large_memory_bytes); -// If memory tracker value is inaccurate, BE will crash. usually used in test environments, default value is false. +// The actual meaning of this parameter is `debug_memory`. +// 1. crash in memory tracker inaccurate, if memory tracker value is inaccurate, BE will crash. +// usually used in test environments, default value is false. +// 2. print more memory logs. DECLARE_mBool(crash_in_memory_tracker_inaccurate); // default is true. if any memory tracking in Orphan mem tracker will report error. @@ -279,6 +282,8 @@ DECLARE_mInt32(max_download_speed_kbps); DECLARE_mInt32(download_low_speed_limit_kbps); // download low speed time(seconds) DECLARE_mInt32(download_low_speed_time); +// whether to download small files in batch. +DECLARE_mBool(enable_batch_download); // deprecated, use env var LOG_DIR in be.conf DECLARE_String(sys_log_dir); @@ -465,6 +470,7 @@ DECLARE_mDouble(base_compaction_min_data_ratio); DECLARE_mInt64(base_compaction_dup_key_max_file_size_mbytes); DECLARE_Bool(enable_skip_tablet_compaction); +DECLARE_mInt32(skip_tablet_compaction_second); // output rowset of cumulative compaction total disk size exceed this config size, // this rowset will be given to base compaction, unit is m byte. DECLARE_mInt64(compaction_promotion_size_mbytes); @@ -1470,6 +1476,7 @@ DECLARE_mBool(enable_pipeline_task_leakage_detect); // MB DECLARE_Int32(query_cache_size); +DECLARE_Bool(force_regenerate_rowsetid_on_start_error); DECLARE_mBool(enable_delete_bitmap_merge_on_compaction); diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index 27fbfb71d7f516..ce2a6878dba034 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -27,17 +27,13 @@ // IWYU pragma: no_include #include #include -#include #include #include -#include -#include // IWYU pragma: no_include #include // IWYU pragma: keep #include #include -#include #include #include "cloud/config.h" @@ -45,30 +41,23 @@ #include "common/logging.h" #include "common/status.h" #include "olap/memtable_memory_limiter.h" -#include "olap/options.h" #include "olap/storage_engine.h" #include "olap/tablet_manager.h" #include "runtime/be_proc_monitor.h" -#include "runtime/client_cache.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/memory/global_memory_arbitrator.h" -#include "runtime/memory/mem_tracker.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/memory/memory_reclamation.h" +#include "runtime/process_profile.h" #include "runtime/runtime_query_statistics_mgr.h" #include "runtime/workload_group/workload_group_manager.h" #include "util/algorithm_util.h" -#include "util/cpu_info.h" -#include "util/debug_util.h" -#include "util/disk_info.h" #include "util/doris_metrics.h" #include "util/mem_info.h" #include "util/metrics.h" -#include "util/network_util.h" #include "util/perf_counters.h" #include "util/system_metrics.h" -#include "util/thrift_util.h" #include "util/time.h" namespace doris { @@ -233,9 +222,8 @@ void refresh_memory_state_after_memory_change() { if (abs(last_print_proc_mem - PerfCounters::get_vm_rss()) > 268435456) { last_print_proc_mem = PerfCounters::get_vm_rss(); doris::MemTrackerLimiter::clean_tracker_limiter_group(); - doris::MemTrackerLimiter::enable_print_log_process_usage(); - // Refresh mem tracker each type counter. - doris::MemTrackerLimiter::refresh_global_counter(); + doris::ProcessProfile::instance()->memory_profile()->enable_print_log_process_usage(); + doris::ProcessProfile::instance()->memory_profile()->refresh_memory_overview_profile(); LOG(INFO) << doris::GlobalMemoryArbitrator:: process_mem_log_str(); // print mem log when memory state by 256M } @@ -339,10 +327,12 @@ void Daemon::memory_gc_thread() { memory_full_gc_sleep_time_ms = memory_gc_sleep_time_ms; memory_minor_gc_sleep_time_ms = memory_gc_sleep_time_ms; LOG(INFO) << fmt::format("[MemoryGC] start full GC, {}.", mem_info); - doris::MemTrackerLimiter::print_log_process_usage(); + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); if (doris::MemoryReclamation::process_full_gc(std::move(mem_info))) { // If there is not enough memory to be gc, the process memory usage will not be printed in the next continuous gc. - doris::MemTrackerLimiter::enable_print_log_process_usage(); + doris::ProcessProfile::instance() + ->memory_profile() + ->enable_print_log_process_usage(); } } else if (memory_minor_gc_sleep_time_ms <= 0 && (sys_mem_available < doris::MemInfo::sys_mem_available_warning_water_mark() || @@ -352,9 +342,11 @@ void Daemon::memory_gc_thread() { doris::GlobalMemoryArbitrator::process_soft_limit_exceeded_errmsg_str(); memory_minor_gc_sleep_time_ms = memory_gc_sleep_time_ms; LOG(INFO) << fmt::format("[MemoryGC] start minor GC, {}.", mem_info); - doris::MemTrackerLimiter::print_log_process_usage(); + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); if (doris::MemoryReclamation::process_minor_gc(std::move(mem_info))) { - doris::MemTrackerLimiter::enable_print_log_process_usage(); + doris::ProcessProfile::instance() + ->memory_profile() + ->enable_print_log_process_usage(); } } else { if (memory_full_gc_sleep_time_ms > 0) { diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index 1b329f76ff6849..85199f2a0be6f6 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -128,7 +128,6 @@ Status SchemaScanner::get_next_block_async(RuntimeState* state) { SCOPED_ATTACH_TASK(state); _dependency->block(); _async_thread_running = true; - _finish_dependency->block(); if (!_opened) { _data_block = vectorized::Block::create_unique(); _init_block(_data_block.get()); @@ -144,9 +143,6 @@ Status SchemaScanner::get_next_block_async(RuntimeState* state) { _eos = eos; _async_thread_running = false; _dependency->set_ready(); - if (eos) { - _finish_dependency->set_ready(); - } })); return Status::OK(); } diff --git a/be/src/exec/schema_scanner.h b/be/src/exec/schema_scanner.h index da61d58b943fc4..35fc051511d8b6 100644 --- a/be/src/exec/schema_scanner.h +++ b/be/src/exec/schema_scanner.h @@ -108,11 +108,7 @@ class SchemaScanner { // factory function static std::unique_ptr create(TSchemaTableType::type type); TSchemaTableType::type type() const { return _schema_table_type; } - void set_dependency(std::shared_ptr dep, - std::shared_ptr fin_dep) { - _dependency = dep; - _finish_dependency = fin_dep; - } + void set_dependency(std::shared_ptr dep) { _dependency = dep; } Status get_next_block_async(RuntimeState* state); protected: @@ -141,7 +137,6 @@ class SchemaScanner { RuntimeProfile::Counter* _fill_block_timer = nullptr; std::shared_ptr _dependency = nullptr; - std::shared_ptr _finish_dependency = nullptr; std::unique_ptr _data_block; AtomicStatus _scanner_status; diff --git a/be/src/exec/schema_scanner/schema_columns_scanner.cpp b/be/src/exec/schema_scanner/schema_columns_scanner.cpp index 8325a7f5dc4f2d..b60dfc3d203f89 100644 --- a/be/src/exec/schema_scanner/schema_columns_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_columns_scanner.cpp @@ -450,7 +450,19 @@ Status SchemaColumnsScanner::_fill_block_impl(vectorized::Block* block) { RETURN_IF_ERROR(fill_dest_column_for_range(block, 4, datas)); } // COLUMN_DEFAULT - { RETURN_IF_ERROR(fill_dest_column_for_range(block, 5, null_datas)); } + { + std::vector strs(columns_num); + for (int i = 0; i < columns_num; ++i) { + if (_desc_result.columns[i].columnDesc.__isset.defaultValue) { + strs[i] = StringRef(_desc_result.columns[i].columnDesc.defaultValue.c_str(), + _desc_result.columns[i].columnDesc.defaultValue.length()); + datas[i] = strs.data() + i; + } else { + datas[i] = nullptr; + } + } + RETURN_IF_ERROR(fill_dest_column_for_range(block, 5, datas)); + } // IS_NULLABLE { StringRef str_yes = StringRef("YES", 3); diff --git a/be/src/gutil/strings/stringpiece.h b/be/src/gutil/strings/stringpiece.h index 38e36a27099279..7a4ebabbf098e7 100644 --- a/be/src/gutil/strings/stringpiece.h +++ b/be/src/gutil/strings/stringpiece.h @@ -149,6 +149,12 @@ class StringPiece { assert(length <= static_cast(std::numeric_limits::max())); length_ = static_cast(length); } + StringPiece(std::string_view view) // NOLINT(runtime/explicit) + : ptr_(view.data()), length_(0) { + size_t length = view.size(); + assert(length <= static_cast(std::numeric_limits::max())); + length_ = static_cast(length); + } StringPiece(const char* offset, int len) : ptr_(offset), length_(len) { assert(len >= 0); } // Substring of another StringPiece. diff --git a/be/src/http/action/batch_download_action.cpp b/be/src/http/action/batch_download_action.cpp new file mode 100644 index 00000000000000..d486883e90be28 --- /dev/null +++ b/be/src/http/action/batch_download_action.cpp @@ -0,0 +1,216 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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 "http/action/batch_download_action.h" + +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/status.h" +#include "gutil/strings/split.h" +#include "http/http_channel.h" +#include "http/http_method.h" +#include "http/http_request.h" +#include "http/utils.h" +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" +#include "util/security.h" + +namespace doris { +namespace { +const std::string CHECK_PARAMETER = "check"; +const std::string LIST_PARAMETER = "list"; +const std::string DIR_PARAMETER = "dir"; +const std::string TOKEN_PARAMETER = "token"; +} // namespace + +BatchDownloadAction::BatchDownloadAction( + ExecEnv* exec_env, std::shared_ptr rate_limit_group, + const std::vector& allow_dirs) + : HttpHandlerWithAuth(exec_env), _rate_limit_group(std::move(rate_limit_group)) { + for (const auto& dir : allow_dirs) { + std::string p; + Status st = io::global_local_filesystem()->canonicalize(dir, &p); + if (!st.ok()) { + continue; + } + _allow_paths.emplace_back(std::move(p)); + } +} + +void BatchDownloadAction::handle(HttpRequest* req) { + if (VLOG_CRITICAL_IS_ON) { + VLOG_CRITICAL << "accept one batch download request " << req->debug_string(); + } + + if (req->param(CHECK_PARAMETER) == "true") { + // For API support check + HttpChannel::send_reply(req, "OK"); + return; + } + + // Get 'dir' parameter, then assembly file absolute path + const std::string& dir_path = req->param(DIR_PARAMETER); + if (dir_path.empty()) { + std::string error_msg = + std::string("parameter " + DIR_PARAMETER + " not specified in url."); + LOG(WARNING) << "handle batch download request: " << error_msg + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg); + return; + } + + if (dir_path.find("..") != std::string::npos) { + std::string error_msg = "Not allowed to read relative path: " + dir_path; + LOG(WARNING) << "handle batch download request: " << error_msg + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::FORBIDDEN, error_msg); + return; + } + + Status status; + if (config::enable_token_check) { + status = _check_token(req); + if (!status.ok()) { + std::string error_msg = status.to_string(); + if (status.is()) { + HttpChannel::send_reply(req, HttpStatus::UNAUTHORIZED, error_msg); + return; + } else { + HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, error_msg); + return; + } + } + } + + status = _check_path_is_allowed(dir_path); + if (!status.ok()) { + std::string error_msg = status.to_string(); + if (status.is() || status.is()) { + HttpChannel::send_reply(req, HttpStatus::NOT_FOUND, error_msg); + return; + } else if (status.is()) { + HttpChannel::send_reply(req, HttpStatus::UNAUTHORIZED, error_msg); + return; + } else { + HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, error_msg); + return; + } + } + + bool is_dir = false; + status = io::global_local_filesystem()->is_directory(dir_path, &is_dir); + if (!status.ok()) { + LOG(WARNING) << "handle batch download request: " << status.to_string() + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, status.to_string()); + return; + } + + if (!is_dir) { + std::string error_msg = fmt::format("The requested path is not a directory: {}", dir_path); + LOG(WARNING) << "handle batch download request: " << error_msg + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg); + return; + } + + _handle(req, dir_path); + + VLOG_CRITICAL << "deal with batch download request finished! "; +} + +void BatchDownloadAction::_handle(HttpRequest* req, const std::string& dir_path) { + bool is_list_request = req->param(LIST_PARAMETER) == "true"; + if (is_list_request) { + // return the list of files in the specified directory + bool is_acquire_filesize = true; + do_dir_response(dir_path, req, is_acquire_filesize); + } else { + _handle_batch_download(req, dir_path); + } +} + +void BatchDownloadAction::_handle_batch_download(HttpRequest* req, const std::string& dir_path) { + std::vector files = + strings::Split(req->get_request_body(), "\n", strings::SkipWhitespace()); + if (files.empty()) { + std::string error_msg = "No file specified in request body."; + LOG(WARNING) << "handle batch download request: " << error_msg + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg); + return; + } + + if (files.size() > 64) { + std::string error_msg = + "The number of files to download in a batch should be less than 64."; + LOG(WARNING) << "handle batch download request: " << error_msg + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg); + return; + } + + for (const auto& file : files) { + if (file.find('/') != std::string::npos) { + std::string error_msg = + fmt::format("Not allowed to read relative path: {}, dir: {}", file, dir_path); + LOG(WARNING) << "handle batch download request: " << error_msg + << ", url: " << mask_token(req->uri()); + HttpChannel::send_reply(req, HttpStatus::FORBIDDEN, error_msg); + return; + } + } + + HttpChannel::send_files(req, dir_path, std::move(files)); +} + +Status BatchDownloadAction::_check_token(HttpRequest* req) { + const std::string& token_str = req->param(TOKEN_PARAMETER); + if (token_str.empty()) { + LOG(WARNING) << "token is not specified in request. url: " << mask_token(req->uri()); + return Status::NotAuthorized("token is not specified."); + } + + const std::string& local_token = _exec_env->token(); + if (token_str != local_token) { + LOG(WARNING) << "invalid download token: " << mask_token(token_str) + << ", local token: " << mask_token(local_token) + << ", url: " << mask_token(req->uri()); + return Status::NotAuthorized("invalid token {}", mask_token(token_str)); + } + + return Status::OK(); +} + +Status BatchDownloadAction::_check_path_is_allowed(const std::string& file_path) { + std::string canonical_file_path; + RETURN_IF_ERROR(io::global_local_filesystem()->canonicalize(file_path, &canonical_file_path)); + for (auto& allow_path : _allow_paths) { + if (io::LocalFileSystem::contain_path(allow_path, canonical_file_path)) { + return Status::OK(); + } + } + + return Status::NotAuthorized("file path is not allowed: {}", canonical_file_path); +} + +} // end namespace doris diff --git a/be/src/http/action/batch_download_action.h b/be/src/http/action/batch_download_action.h new file mode 100644 index 00000000000000..f0b7e3576b9937 --- /dev/null +++ b/be/src/http/action/batch_download_action.h @@ -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. + +#pragma once + +#include +#include + +#include "common/status.h" +#include "http/http_handler.h" +#include "http/http_handler_with_auth.h" +#include "util/threadpool.h" + +struct bufferevent_rate_limit_group; + +namespace doris { + +class ExecEnv; +class HttpRequest; + +// A simple handler that serves incoming HTTP requests of batching file-download to send their +// respective HTTP responses. +// +// We use parameter named 'dir' to specify the static resource path, it is an absolute path. +// +// In HEAD request, then this handler will return the list of files in the specified directory. +// +// In GET request, the file names to download are specified in the request body as a list of strings, +// separated by '\n'. To avoid cost resource, the maximum number of files to download in a batch is 64. +class BatchDownloadAction : public HttpHandlerWithAuth { +public: + BatchDownloadAction(ExecEnv* exec_env, + std::shared_ptr rate_limit_group, + const std::vector& allow_dirs); + + ~BatchDownloadAction() override = default; + + void handle(HttpRequest* req) override; + +private: + Status _check_token(HttpRequest* req); + Status _check_path_is_allowed(const std::string& path); + + void _handle(HttpRequest* req, const std::string& dir_path); + void _handle_batch_download(HttpRequest* req, const std::string& dir_path); + + std::vector _allow_paths; + std::shared_ptr _rate_limit_group; +}; + +} // end namespace doris diff --git a/be/src/http/action/download_binlog_action.cpp b/be/src/http/action/download_binlog_action.cpp index 54701c5e463481..372f840401c4ad 100644 --- a/be/src/http/action/download_binlog_action.cpp +++ b/be/src/http/action/download_binlog_action.cpp @@ -21,11 +21,9 @@ #include #include -#include #include #include #include -#include #include "common/config.h" #include "common/logging.h" @@ -34,7 +32,6 @@ #include "http/utils.h" #include "io/fs/local_file_system.h" #include "olap/storage_engine.h" -#include "olap/tablet.h" #include "olap/tablet_manager.h" #include "runtime/exec_env.h" diff --git a/be/src/http/default_path_handlers.cpp b/be/src/http/default_path_handlers.cpp index 2ece1e3fdcd20a..04e1121cab63ba 100644 --- a/be/src/http/default_path_handlers.cpp +++ b/be/src/http/default_path_handlers.cpp @@ -38,13 +38,9 @@ #include #include "common/config.h" -#include "gutil/strings/numbers.h" -#include "gutil/strings/substitute.h" #include "http/action/tablets_info_action.h" #include "http/web_page_handler.h" -#include "runtime/memory/global_memory_arbitrator.h" -#include "runtime/memory/mem_tracker.h" -#include "runtime/memory/mem_tracker_limiter.h" +#include "runtime/process_profile.h" #include "util/easy_json.h" #include "util/mem_info.h" #include "util/perf_counters.h" @@ -97,16 +93,51 @@ void config_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* (*output) << ""; } -// Registered to handle "/memz", and prints out memory allocation statistics. -void mem_usage_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) { - (*output) << "
"
-              << "Mem Limit: " << PrettyPrinter::print(MemInfo::mem_limit(), TUnit::BYTES)
+void memory_info_handler(std::stringstream* output) {
+    (*output) << "

Memory Info

\n"; + (*output) << "
";
+    (*output) << "

Memory Documents

\n" + << "Memory Management Overview\n" + << "Memory Issue FAQ\n" + << "\n---\n\n"; + + (*output) << "

Memory Properties

\n" + << "System Physical Mem: " + << PrettyPrinter::print(MemInfo::physical_mem(), TUnit::BYTES) << std::endl + << "System Page Size: " << MemInfo::get_page_size() << std::endl + << "Mem Limit: " << MemInfo::mem_limit_str() << std::endl + << "Soft Mem Limit: " << MemInfo::soft_mem_limit_str() << std::endl + << "System Mem Available Low Water Mark: " + << PrettyPrinter::print(MemInfo::sys_mem_available_low_water_mark(), TUnit::BYTES) + << std::endl + << "System Mem Available Warning Water Mark: " + << PrettyPrinter::print(MemInfo::sys_mem_available_warning_water_mark(), TUnit::BYTES) << std::endl - << "Physical Mem From Perf: " - << PrettyPrinter::print(PerfCounters::get_vm_rss(), TUnit::BYTES) << std::endl - << "
"; + << "Cgroup Mem Limit: " + << PrettyPrinter::print(MemInfo::cgroup_mem_limit(), TUnit::BYTES) << std::endl + << "Cgroup Mem Usage: " + << PrettyPrinter::print(MemInfo::cgroup_mem_usage(), TUnit::BYTES) << std::endl + << "Cgroup Mem Refresh State: " << MemInfo::cgroup_mem_refresh_state() << std::endl + << "\n---\n\n"; + + (*output) << "

Memory Option Settings

\n"; + { + std::lock_guard lock(*config::get_mutable_string_config_lock()); + for (const auto& it : *(config::full_conf_map)) { + if (it.first.find("memory") != std::string::npos || + it.first.find("cache") != std::string::npos || + it.first.find("mem") != std::string::npos) { + (*output) << it.first << "=" << it.second << std::endl; + } + } + } + (*output) << "\n---\n\n"; - (*output) << "
";
+    (*output) << "

Jemalloc Profiles

\n"; #if defined(ADDRESS_SANITIZER) || defined(LEAK_SANITIZER) || defined(THREAD_SANITIZER) (*output) << "Memory tracking is not available with address sanitizer builds."; #elif defined(USE_JEMALLOC) @@ -117,15 +148,59 @@ void mem_usage_handler(const WebPageHandler::ArgumentMap& args, std::stringstrea }; jemalloc_stats_print(write_cb, &tmp, "a"); boost::replace_all(tmp, "\n", "
"); - (*output) << tmp << "
"; + (*output) << tmp; #else char buf[2048]; MallocExtension::instance()->GetStats(buf, 2048); // Replace new lines with
for html std::string tmp(buf); boost::replace_all(tmp, "\n", "
"); - (*output) << tmp << "
"; + (*output) << tmp; #endif + (*output) << ""; +} + +// Registered to handle "/profile". +void process_profile_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) { + (*output) << "

Copy Process Profile To Clipboard (拷贝 Process Profile 到剪切板)

"; + (*output) << "" << std::endl; + (*output) << "" << std::endl; + + doris::ProcessProfile::instance()->refresh_profile(); + + (*output) << "
" << std::endl; + (*output) << "

Process Profile

" << std::endl; + (*output) << "
"
+              << doris::ProcessProfile::instance()->print_process_profile_no_root() << "
" + << "\n\n---\n\n"; + memory_info_handler(output); + + // TODO, expect more information about process status, CPU, IO, etc. + + (*output) << "
" << std::endl; } void display_tablets_callback(const WebPageHandler::ArgumentMap& args, EasyJson* ej) { @@ -141,76 +216,8 @@ void display_tablets_callback(const WebPageHandler::ArgumentMap& args, EasyJson* // Registered to handle "/mem_tracker", and prints out memory tracker information. void mem_tracker_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) { - (*output) << "

Memory usage by subsystem

\n"; - std::vector snapshots; - auto iter = args.find("type"); - if (iter != args.end()) { - if (iter->second == "global") { - MemTrackerLimiter::make_type_snapshots(&snapshots, MemTrackerLimiter::Type::GLOBAL); - } else if (iter->second == "query") { - MemTrackerLimiter::make_type_snapshots(&snapshots, MemTrackerLimiter::Type::QUERY); - } else if (iter->second == "load") { - MemTrackerLimiter::make_type_snapshots(&snapshots, MemTrackerLimiter::Type::LOAD); - } else if (iter->second == "compaction") { - MemTrackerLimiter::make_type_snapshots(&snapshots, MemTrackerLimiter::Type::COMPACTION); - } else if (iter->second == "schema_change") { - MemTrackerLimiter::make_type_snapshots(&snapshots, - MemTrackerLimiter::Type::SCHEMA_CHANGE); - } else if (iter->second == "other") { - MemTrackerLimiter::make_type_snapshots(&snapshots, MemTrackerLimiter::Type::OTHER); - } else if (iter->second == "reserved_memory") { - MemTrackerLimiter::make_all_reserved_trackers_snapshots(&snapshots); - } else if (iter->second == "all") { - MemTrackerLimiter::make_all_memory_state_snapshots(&snapshots); - } - } else { - (*output) << "

*Notice:

\n"; - (*output) << "

1. MemTracker only counts the memory on part of the main execution " - "path, " - "which is usually less than the real process memory.

\n"; - (*output) << "

2. each `type` is the sum of a set of tracker values, " - "`sum of all trackers` is the sum of all trackers of all types, .

\n"; - (*output) << "

3. `process resident memory` is the physical memory of the process, " - "from /proc VmRSS VmHWM.

\n"; - (*output) << "

4. `process virtual memory` is the virtual memory of the process, " - "from /proc VmSize VmPeak.

\n"; - (*output) << "

5.`/mem_tracker?type=` to view the memory details of each " - "type, for example, `/mem_tracker?type=query` will list the memory of all " - "queries; " - "`/mem_tracker?type=global` will list the memory of all Cache, metadata and " - "other " - "global life cycles.

\n"; - (*output) << "

see documentation for details."; - MemTrackerLimiter::make_process_snapshots(&snapshots); - } - - (*output) << "\n"; - (*output) << "" - "" - "" - "" - "" - "" - "" - "" - ""; - (*output) << "\n"; - for (const auto& item : snapshots) { - string limit_str = item.limit == -1 ? "none" : AccurateItoaKMGT(item.limit); - string current_consumption_normalize = AccurateItoaKMGT(item.cur_consumption); - string peak_consumption_normalize = AccurateItoaKMGT(item.peak_consumption); - (*output) << strings::Substitute( - "\n", - item.type, item.label, limit_str, item.cur_consumption, - current_consumption_normalize, item.peak_consumption, peak_consumption_normalize); - } - (*output) << "
TypeLabelLimitCurrent Consumption(Bytes)Current Consumption(Normalize)Peak Consumption(Bytes)Peak Consumption(Normalize)
$0$1$2$3$4$5$6
\n"; + (*output) << "

mem_tracker webpage has been offline, please click Process Profile, see MemoryProfile and Memory Info

\n"; } void heap_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) { @@ -394,14 +401,10 @@ void add_default_path_handlers(WebPageHandler* web_page_handler) { web_page_handler->register_page("/varz", "Configs", config_handler, true /* is_on_nav_bar */); } - web_page_handler->register_page("/memz", "Memory", mem_usage_handler, true /* is_on_nav_bar */); - web_page_handler->register_page( - "/mem_tracker", "MemTracker", - [](auto&& PH1, auto&& PH2) { - return mem_tracker_handler(std::forward(PH1), - std::forward(PH2)); - }, - true /* is_on_nav_bar */); + web_page_handler->register_page("/profile", "Process Profile", process_profile_handler, + true /* is_on_nav_bar */); + web_page_handler->register_page("/mem_tracker", "MemTracker", mem_tracker_handler, + true /* is_on_nav_bar */); web_page_handler->register_page("/heap", "Heap Profile", heap_handler, true /* is_on_nav_bar */); web_page_handler->register_page("/cpu", "CPU Profile", cpu_handler, true /* is_on_nav_bar */); diff --git a/be/src/http/http_channel.cpp b/be/src/http/http_channel.cpp index 96679195316dac..312f1ab9286909 100644 --- a/be/src/http/http_channel.cpp +++ b/be/src/http/http_channel.cpp @@ -20,8 +20,8 @@ #include #include #include +#include -#include #include #include #include @@ -57,7 +57,7 @@ void HttpChannel::send_reply(HttpRequest* request, HttpStatus status) { } void HttpChannel::send_reply(HttpRequest* request, HttpStatus status, const std::string& content) { - auto evb = evbuffer_new(); + auto* evb = evbuffer_new(); std::string compressed_content; if (compress_content(request->header(HttpHeaders::ACCEPT_ENCODING), content, &compressed_content)) { @@ -72,7 +72,7 @@ void HttpChannel::send_reply(HttpRequest* request, HttpStatus status, const std: void HttpChannel::send_file(HttpRequest* request, int fd, size_t off, size_t size, bufferevent_rate_limit_group* rate_limit_group) { - auto evb = evbuffer_new(); + auto* evb = evbuffer_new(); evbuffer_add_file(evb, fd, off, size); auto* evhttp_request = request->get_evhttp_request(); if (rate_limit_group) { @@ -84,6 +84,56 @@ void HttpChannel::send_file(HttpRequest* request, int fd, size_t off, size_t siz evbuffer_free(evb); } +void HttpChannel::send_files(HttpRequest* request, const std::string& root_dir, + std::vector local_files, + bufferevent_rate_limit_group* rate_limit_group) { + if (rate_limit_group) { + auto* evhttp_request = request->get_evhttp_request(); + auto* evhttp_connection = evhttp_request_get_connection(evhttp_request); + auto* buffer_event = evhttp_connection_get_bufferevent(evhttp_connection); + bufferevent_add_to_rate_limit_group(buffer_event, rate_limit_group); + } + + send_files(request, root_dir, std::move(local_files)); +} + +void HttpChannel::send_files(HttpRequest* request, const std::string& root_dir, + std::vector local_files) { + std::unique_ptr evb(evbuffer_new(), &evbuffer_free); + for (const std::string& file : local_files) { + std::string file_path = fmt::format("{}/{}", root_dir, file); + int fd = open(file_path.c_str(), O_RDONLY); + if (fd < 0) { + std::string error_msg = "Failed to open file: " + file_path; + LOG(WARNING) << "http channel send files: " << error_msg; + HttpChannel::send_reply(request, HttpStatus::NOT_FOUND, error_msg); + return; + } + struct stat st; + auto res = fstat(fd, &st); + if (res < 0) { + close(fd); + std::string error_msg = "Failed to open file: " + file_path; + LOG(WARNING) << "http channel send files: " << error_msg; + HttpChannel::send_reply(request, HttpStatus::NOT_FOUND, error_msg); + return; + } + + int64_t file_size = st.st_size; + VLOG_DEBUG << "http channel send file " << file_path << ", size: " << file_size; + + evbuffer_add_printf(evb.get(), "File-Name: %s\r\n", file.c_str()); + evbuffer_add_printf(evb.get(), "Content-Length: %ld\r\n", file_size); + evbuffer_add_printf(evb.get(), "\r\n"); + if (file_size > 0) { + evbuffer_add_file(evb.get(), fd, 0, file_size); + } + } + + evhttp_send_reply(request->get_evhttp_request(), HttpStatus::OK, + default_reason(HttpStatus::OK).c_str(), evb.get()); +} + bool HttpChannel::compress_content(const std::string& accept_encoding, const std::string& input, std::string* output) { // Don't bother compressing empty content. diff --git a/be/src/http/http_channel.h b/be/src/http/http_channel.h index ee1e6c0888f1d3..0d5e5d4260af8c 100644 --- a/be/src/http/http_channel.h +++ b/be/src/http/http_channel.h @@ -20,6 +20,7 @@ #include #include +#include #include "http/http_status.h" @@ -47,6 +48,13 @@ class HttpChannel { static void send_file(HttpRequest* request, int fd, size_t off, size_t size, bufferevent_rate_limit_group* rate_limit_group = nullptr); + static void send_files(HttpRequest* request, const std::string& root_dir, + std::vector local_files, + bufferevent_rate_limit_group* rate_limit_group); + + static void send_files(HttpRequest* request, const std::string& root_dir, + std::vector local_files); + static bool compress_content(const std::string& accept_encoding, const std::string& input, std::string* output); }; diff --git a/be/src/http/http_client.cpp b/be/src/http/http_client.cpp index fc4c997fce8397..767377cea3f365 100644 --- a/be/src/http/http_client.cpp +++ b/be/src/http/http_client.cpp @@ -24,14 +24,225 @@ #include #include "common/config.h" +#include "common/status.h" #include "http/http_headers.h" -#include "http/http_status.h" #include "runtime/exec_env.h" #include "util/security.h" #include "util/stack_util.h" namespace doris { +class MultiFileSplitter { +public: + MultiFileSplitter(std::string local_dir, std::unordered_set expected_files) + : _local_dir_path(std::move(local_dir)), _expected_files(std::move(expected_files)) {} + ~MultiFileSplitter() { + if (_fd >= 0) { + close(_fd); + } + + if (!_status.ok() && !downloaded_files.empty()) { + LOG(WARNING) << "download files to " << _local_dir_path << " failed, try remove the " + << downloaded_files.size() << " downloaded files"; + for (const auto& file : downloaded_files) { + remove(file.c_str()); + } + } + } + + bool append(const char* data, size_t length) { + // Already failed. + if (!_status.ok()) { + return false; + } + + std::string buf; + if (!_buffer.empty()) { + buf.swap(_buffer); + buf.append(data, length); + data = buf.data(); + length = buf.size(); + } + return append_inner(data, length); + } + + Status finish() { + if (_status.ok()) { + _status = finish_inner(); + } + + return _status; + } + +private: + bool append_inner(const char* data, size_t length) { + while (length > 0) { + int consumed = 0; + if (_is_reading_header) { + consumed = parse_header(data, length); + } else { + consumed = append_file(data, length); + } + + if (consumed < 0) { + return false; + } + + DCHECK(consumed <= length); + data += consumed; + length -= consumed; + } + return true; + } + + int parse_header(const char* data, size_t length) { + DCHECK(_fd < 0); + + std::string_view buf(data, length); + size_t pos = buf.find("\r\n\r\n"); + if (pos == std::string::npos) { + _buffer.append(data, length); + return static_cast(length); + } + + // header already read. + _is_reading_header = false; + + bool has_file_name = false; + bool has_file_size = false; + std::string_view header = buf.substr(0, pos); + std::vector headers = + strings::Split(header, "\r\n", strings::SkipWhitespace()); + for (auto& s : headers) { + size_t header_pos = s.find(':'); + if (header_pos == std::string::npos) { + continue; + } + std::string_view header_view(s); + std::string_view key = header_view.substr(0, header_pos); + std::string_view value = header_view.substr(header_pos + 1); + if (value.starts_with(' ')) { + value.remove_prefix(std::min(value.find_first_not_of(' '), value.size())); + } + if (key == "File-Name") { + _file_name = value; + has_file_name = true; + } else if (key == "Content-Length") { + auto res = std::from_chars(value.data(), value.data() + value.size(), _file_size); + if (res.ec != std::errc()) { + std::string error_msg = fmt::format("invalid content length: {}", value); + LOG(WARNING) << "download files to " << _local_dir_path + << "failed, err=" << error_msg; + _status = Status::HttpError(std::move(error_msg)); + return -1; + } + has_file_size = true; + } + } + + if (!has_file_name || !has_file_size) { + std::string error_msg = + fmt::format("invalid multi part header, has file name: {}, has file size: {}", + has_file_name, has_file_size); + LOG(WARNING) << "download files to " << _local_dir_path << "failed, err=" << error_msg; + _status = Status::HttpError(std::move(error_msg)); + return -1; + } + + if (!_expected_files.contains(_file_name)) { + std::string error_msg = fmt::format("unexpected file: {}", _file_name); + LOG(WARNING) << "download files to " << _local_dir_path << "failed, err=" << error_msg; + _status = Status::HttpError(std::move(error_msg)); + return -1; + } + + VLOG_DEBUG << "receive file " << _file_name << ", size " << _file_size; + + _written_size = 0; + _local_file_path = fmt::format("{}/{}", _local_dir_path, _file_name); + _fd = open(_local_file_path.c_str(), O_WRONLY | O_CREAT | O_TRUNC, 0644); + if (_fd < 0) { + std::string error_msg = "fail to open file to write: " + _local_file_path; + LOG(WARNING) << "download files to " << _local_dir_path << "failed, err=" << error_msg; + _status = Status::IOError(std::move(error_msg)); + return -1; + } + downloaded_files.push_back(_local_file_path); + + return static_cast(pos + 4); + } + + int append_file(const char* data, size_t length) { + DCHECK(_fd >= 0); + DCHECK(_file_size >= _written_size); + + size_t write_size = std::min(length, _file_size - _written_size); + if (write_size > 0 && write(_fd, data, write_size) < 0) { + auto msg = fmt::format("write file failed, file={}, error={}", _local_file_path, + strerror(errno)); + LOG(WARNING) << "download files to " << _local_dir_path << "failed, err=" << msg; + _status = Status::HttpError(std::move(msg)); + return -1; + } + + _written_size += write_size; + if (_written_size == _file_size) { + // This file has been downloaded, switch to the next one. + switchToNextFile(); + } + + return write_size; + } + + Status finish_inner() { + if (!_is_reading_header && _written_size == _file_size) { + switchToNextFile(); + } + + if (_fd >= 0) { + // This file is not completely downloaded. + close(_fd); + _fd = -1; + auto error_msg = fmt::format("file {} is not completely downloaded", _local_file_path); + LOG(WARNING) << "download files to " << _local_dir_path << "failed, err=" << error_msg; + return Status::HttpError(std::move(error_msg)); + } + + if (!_expected_files.empty()) { + auto error_msg = fmt::format("not all files are downloaded, {} missing files", + _expected_files.size()); + LOG(WARNING) << "download files to " << _local_dir_path << "failed, err=" << error_msg; + return Status::HttpError(std::move(error_msg)); + } + + downloaded_files.clear(); + return Status::OK(); + } + + void switchToNextFile() { + DCHECK(_fd >= 0); + DCHECK(_written_size == _file_size); + + close(_fd); + _fd = -1; + _expected_files.erase(_file_name); + _is_reading_header = true; + } + + const std::string _local_dir_path; + std::string _buffer; + std::unordered_set _expected_files; + Status _status; + + bool _is_reading_header = true; + int _fd = -1; + std::string _local_file_path; + std::string _file_name; + size_t _file_size = 0; + size_t _written_size = 0; + std::vector downloaded_files; +}; + static const char* header_error_msg(CURLHcode code) { switch (code) { case CURLHE_OK: @@ -174,6 +385,12 @@ void HttpClient::set_method(HttpMethod method) { } } +void HttpClient::set_speed_limit() { + curl_easy_setopt(_curl, CURLOPT_LOW_SPEED_LIMIT, config::download_low_speed_limit_kbps * 1024); + curl_easy_setopt(_curl, CURLOPT_LOW_SPEED_TIME, config::download_low_speed_time); + curl_easy_setopt(_curl, CURLOPT_MAX_RECV_SPEED_LARGE, config::max_download_speed_kbps * 1024); +} + size_t HttpClient::on_response_data(const void* data, size_t length) { if (*_callback != nullptr) { bool is_continue = (*_callback)(data, length); @@ -184,12 +401,6 @@ size_t HttpClient::on_response_data(const void* data, size_t length) { return length; } -// Status HttpClient::execute_post_request(const std::string& post_data, const std::function& callback = {}) { -// _callback = &callback; -// set_post_body(post_data); -// return execute(callback); -// } - Status HttpClient::execute_post_request(const std::string& payload, std::string* response) { set_method(POST); set_payload(payload); @@ -234,14 +445,8 @@ Status HttpClient::get_content_md5(std::string* md5) const { } Status HttpClient::download(const std::string& local_path) { - // set method to GET set_method(GET); - - // TODO(zc) Move this download speed limit outside to limit download speed - // at system level - curl_easy_setopt(_curl, CURLOPT_LOW_SPEED_LIMIT, config::download_low_speed_limit_kbps * 1024); - curl_easy_setopt(_curl, CURLOPT_LOW_SPEED_TIME, config::download_low_speed_time); - curl_easy_setopt(_curl, CURLOPT_MAX_RECV_SPEED_LARGE, config::max_download_speed_kbps * 1024); + set_speed_limit(); auto fp_closer = [](FILE* fp) { fclose(fp); }; std::unique_ptr fp(fopen(local_path.c_str(), "w"), fp_closer); @@ -270,6 +475,20 @@ Status HttpClient::download(const std::string& local_path) { return status; } +Status HttpClient::download_multi_files(const std::string& local_dir, + const std::unordered_set& expected_files) { + set_speed_limit(); + + MultiFileSplitter splitter(local_dir, expected_files); + auto callback = [&](const void* data, size_t length) { + return splitter.append(reinterpret_cast(data), length); + }; + if (auto s = execute(callback); !s.ok()) { + return s; + } + return splitter.finish(); +} + Status HttpClient::execute(std::string* response) { auto callback = [response](const void* data, size_t length) { response->append((char*)data, length); diff --git a/be/src/http/http_client.h b/be/src/http/http_client.h index c0c8863a9b06d4..a6f2f4fdff514b 100644 --- a/be/src/http/http_client.h +++ b/be/src/http/http_client.h @@ -24,6 +24,7 @@ #include #include #include +#include #include "common/status.h" #include "http/http_headers.h" @@ -81,6 +82,8 @@ class HttpClient { curl_easy_setopt(_curl, CURLOPT_SSL_VERIFYHOST, 0L); } + void set_speed_limit(); + // TODO(zc): support set header // void set_header(const std::string& key, const std::string& value) { // _cntl.http_request().SetHeader(key, value); @@ -141,6 +144,8 @@ class HttpClient { // helper function to download a file, you can call this function to download // a file to local_path Status download(const std::string& local_path); + Status download_multi_files(const std::string& local_dir, + const std::unordered_set& expected_files); Status execute_post_request(const std::string& payload, std::string* response); diff --git a/be/src/http/http_handler_with_auth.cpp b/be/src/http/http_handler_with_auth.cpp index 518b9868de191e..ae5c024e76d093 100644 --- a/be/src/http/http_handler_with_auth.cpp +++ b/be/src/http/http_handler_with_auth.cpp @@ -35,6 +35,7 @@ HttpHandlerWithAuth::HttpHandlerWithAuth(ExecEnv* exec_env, TPrivilegeHier::type : _exec_env(exec_env), _hier(hier), _type(type) {} int HttpHandlerWithAuth::on_header(HttpRequest* req) { + //if u return value isn't 0,u should `send_reply`,Avoid requesting links that never return. TCheckAuthRequest auth_request; TCheckAuthResult auth_result; AuthInfo auth_info; @@ -83,6 +84,11 @@ int HttpHandlerWithAuth::on_header(HttpRequest* req) { #ifndef BE_TEST TNetworkAddress master_addr = _exec_env->cluster_info()->master_fe_addr; + if (master_addr.hostname.empty() || master_addr.port == 0) { + LOG(WARNING) << "Not found master fe, Can't auth API request: " << req->debug_string(); + HttpChannel::send_error(req, HttpStatus::SERVICE_UNAVAILABLE); + return -1; + } { auto status = ThriftRpcHelper::rpc( master_addr.hostname, master_addr.port, @@ -90,6 +96,10 @@ int HttpHandlerWithAuth::on_header(HttpRequest* req) { client->checkAuth(auth_result, auth_request); }); if (!status) { + LOG(WARNING) << "CheckAuth Rpc Fail.Fe Ip:" << master_addr.hostname + << ", Fe port:" << master_addr.port << ".Status:" << status.to_string() + << ".Request: " << req->debug_string(); + HttpChannel::send_error(req, HttpStatus::SERVICE_UNAVAILABLE); return -1; } } @@ -98,6 +108,7 @@ int HttpHandlerWithAuth::on_header(HttpRequest* req) { auth_result.status.status_code = TStatusCode::type::OK; auth_result.status.error_msgs.clear(); } else { + HttpChannel::send_reply(req, HttpStatus::FORBIDDEN); return -1; } #endif diff --git a/be/src/http/utils.cpp b/be/src/http/utils.cpp index f91610476b4dc9..ee7a78113e555a 100644 --- a/be/src/http/utils.cpp +++ b/be/src/http/utils.cpp @@ -23,6 +23,8 @@ #include #include +#include +#include #include #include "common/config.h" @@ -30,6 +32,7 @@ #include "common/status.h" #include "common/utils.h" #include "http/http_channel.h" +#include "http/http_client.h" #include "http/http_common.h" #include "http/http_headers.h" #include "http/http_method.h" @@ -41,10 +44,15 @@ #include "runtime/exec_env.h" #include "util/md5.h" #include "util/path_util.h" +#include "util/security.h" #include "util/url_coding.h" namespace doris { +const uint32_t CHECK_SUPPORT_TIMEOUT = 3; +const uint32_t DOWNLOAD_FILE_MAX_RETRY = 3; +const uint32_t LIST_REMOTE_FILE_TIMEOUT = 15; + std::string encode_basic_auth(const std::string& user, const std::string& passwd) { std::string auth = user + ":" + passwd; std::string encoded_auth; @@ -190,20 +198,26 @@ void do_file_response(const std::string& file_path, HttpRequest* req, HttpChannel::send_file(req, fd, 0, file_size, rate_limit_group); } -void do_dir_response(const std::string& dir_path, HttpRequest* req) { +void do_dir_response(const std::string& dir_path, HttpRequest* req, bool is_acquire_filesize) { bool exists = true; std::vector files; Status st = io::global_local_filesystem()->list(dir_path, true, &files, &exists); if (!st.ok()) { LOG(WARNING) << "Failed to scan dir. " << st; HttpChannel::send_error(req, HttpStatus::INTERNAL_SERVER_ERROR); + return; } + VLOG_DEBUG << "list dir: " << dir_path << ", file count: " << files.size(); + const std::string FILE_DELIMITER_IN_DIR_RESPONSE = "\n"; std::stringstream result; for (auto& file : files) { result << file.file_name << FILE_DELIMITER_IN_DIR_RESPONSE; + if (is_acquire_filesize) { + result << file.file_size << FILE_DELIMITER_IN_DIR_RESPONSE; + } } std::string result_str = result.str(); @@ -221,4 +235,118 @@ bool load_size_smaller_than_wal_limit(int64_t content_length) { return (content_length < 0.8 * max_available_size); } +Status is_support_batch_download(const std::string& endpoint) { + std::string url = fmt::format("http://{}/api/_tablet/_batch_download?check=true", endpoint); + auto check_support_cb = [&url](HttpClient* client) { + RETURN_IF_ERROR(client->init(url)); + client->set_timeout_ms(CHECK_SUPPORT_TIMEOUT * 1000); + client->set_method(HttpMethod::HEAD); + std::string response; + return client->execute(&response); + }; + return HttpClient::execute_with_retry(DOWNLOAD_FILE_MAX_RETRY, 1, check_support_cb); +} + +Status list_remote_files_v2(const std::string& address, const std::string& token, + const std::string& remote_dir, + std::vector>* file_info_list) { + std::string remote_url = + fmt::format("http://{}/api/_tablet/_batch_download?token={}&dir={}&list=true", address, + token, remote_dir); + + std::string file_list_str; + auto list_files_cb = [&](HttpClient* client) { + file_list_str.clear(); + RETURN_IF_ERROR(client->init(remote_url, false)); + client->set_method(HttpMethod::GET); + client->set_timeout_ms(LIST_REMOTE_FILE_TIMEOUT * 1000); + return client->execute(&file_list_str); + }; + Status status = HttpClient::execute_with_retry(DOWNLOAD_FILE_MAX_RETRY, 1, list_files_cb); + if (!status.ok()) { + LOG(WARNING) << "failed to list remote files from " << remote_url + << ", status: " << status.to_string() << ", response: " << file_list_str; + return status; + } + + std::vector file_list = strings::Split(file_list_str, "\n", strings::SkipWhitespace()); + if (file_list.size() % 2 != 0) { + return Status::InternalError("batch download files: invalid file list, size is not even"); + } + + VLOG_DEBUG << "list remote files from " << remote_url + << ", file count: " << file_list.size() / 2; + + for (size_t i = 0; i < file_list.size(); i += 2) { + uint64_t file_size = 0; + try { + file_size = std::stoull(file_list[i + 1]); + } catch (std::exception&) { + return Status::InternalError("batch download files: invalid file size format: " + + file_list[i + 1]); + } + file_info_list->emplace_back(std::move(file_list[i]), file_size); + } + + return Status::OK(); +} + +Status download_files_v2(const std::string& address, const std::string& token, + const std::string& remote_dir, const std::string& local_dir, + const std::vector>& file_info_list) { + std::string remote_url = fmt::format("http://{}/api/_tablet/_batch_download?dir={}&token={}", + address, remote_dir, token); + + size_t batch_file_size = 0; + std::unordered_set expected_files; + std::stringstream ss; + for (const auto& file_info : file_info_list) { + ss << file_info.first << "\n"; + batch_file_size += file_info.second; + expected_files.insert(file_info.first); + } + std::string payload = ss.str(); + + uint64_t estimate_timeout = batch_file_size / config::download_low_speed_limit_kbps / 1024; + if (estimate_timeout < config::download_low_speed_time) { + estimate_timeout = config::download_low_speed_time; + } + + LOG(INFO) << "begin to download files from " << remote_url << " to " << local_dir + << ", file count: " << file_info_list.size() << ", total size: " << batch_file_size + << ", timeout: " << estimate_timeout; + + auto callback = [&](HttpClient* client) -> Status { + RETURN_IF_ERROR(client->init(remote_url, false)); + client->set_method(HttpMethod::POST); + client->set_payload(payload); + client->set_timeout_ms(estimate_timeout * 1000); + RETURN_IF_ERROR(client->download_multi_files(local_dir, expected_files)); + for (auto&& [file_name, file_size] : file_info_list) { + std::string local_file_path = local_dir + "/" + file_name; + + std::error_code ec; + // Check file length + uint64_t local_file_size = std::filesystem::file_size(local_file_path, ec); + if (ec) { + LOG(WARNING) << "download file error: " << ec.message(); + return Status::IOError("can't retrive file_size of {}, due to {}", local_file_path, + ec.message()); + } + if (local_file_size != file_size) { + LOG(WARNING) << "download file length error" + << ", remote_path=" << mask_token(remote_url) + << ", file_name=" << file_name << ", file_size=" << file_size + << ", local_file_size=" << local_file_size; + return Status::InternalError("downloaded file size is not equal"); + } + RETURN_IF_ERROR(io::global_local_filesystem()->permission( + local_file_path, io::LocalFileSystem::PERMS_OWNER_RW)); + } + + return Status::OK(); + }; + return HttpClient::execute_with_retry(DOWNLOAD_FILE_MAX_RETRY, 1, callback); +} + } // namespace doris diff --git a/be/src/http/utils.h b/be/src/http/utils.h index 20be6c0fcd7439..b9abb7c6208efb 100644 --- a/be/src/http/utils.h +++ b/be/src/http/utils.h @@ -40,9 +40,22 @@ void do_file_response(const std::string& dir_path, HttpRequest* req, bufferevent_rate_limit_group* rate_limit_group = nullptr, bool is_acquire_md5 = false); -void do_dir_response(const std::string& dir_path, HttpRequest* req); +void do_dir_response(const std::string& dir_path, HttpRequest* req, + bool is_acquire_filesize = false); std::string get_content_type(const std::string& file_name); bool load_size_smaller_than_wal_limit(int64_t content_length); + +// Whether a backend supports batch download +Status is_support_batch_download(const std::string& address); + +Status list_remote_files_v2(const std::string& address, const std::string& token, + const std::string& remote_dir, + std::vector>* file_info_list); + +Status download_files_v2(const std::string& address, const std::string& token, + const std::string& remote_dir, const std::string& local_dir, + const std::vector>& file_info_list); + } // namespace doris diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index 528ace90e3e7d7..2a59a5158e46c2 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -1093,15 +1093,16 @@ bool BlockFileCache::remove_if_ttl_file_unlock(const UInt128Wrapper& file_key, b _key_to_time.find(file_key) != _key_to_time.end()) { if (!remove_directly) { for (auto& [_, cell] : _files[file_key]) { - if (cell.file_block->cache_type() == FileCacheType::TTL) { - Status st = cell.file_block->update_expiration_time(0); - if (!st.ok()) { - LOG_WARNING("Failed to update expiration time to 0").error(st); - } + if (cell.file_block->cache_type() != FileCacheType::TTL) { + continue; + } + Status st = cell.file_block->update_expiration_time(0); + if (!st.ok()) { + LOG_WARNING("Failed to update expiration time to 0").error(st); } if (cell.file_block->cache_type() == FileCacheType::NORMAL) continue; - auto st = cell.file_block->change_cache_type_between_ttl_and_others( + st = cell.file_block->change_cache_type_between_ttl_and_others( FileCacheType::NORMAL); if (st.ok()) { if (cell.queue_iterator) { diff --git a/be/src/io/fs/file_system.cpp b/be/src/io/fs/file_system.cpp index 3579a5323d9217..e6b5ef7df1a8f5 100644 --- a/be/src/io/fs/file_system.cpp +++ b/be/src/io/fs/file_system.cpp @@ -25,58 +25,70 @@ namespace io { Status FileSystem::create_file(const Path& file, FileWriterPtr* writer, const FileWriterOptions* opts) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(create_file_impl(path, writer, opts)); } Status FileSystem::open_file(const Path& file, FileReaderSPtr* reader, const FileReaderOptions* opts) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(open_file_impl(path, reader, opts)); } Status FileSystem::create_directory(const Path& dir, bool failed_if_exists) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(create_directory_impl(path, failed_if_exists)); } Status FileSystem::delete_file(const Path& file) { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(delete_file_impl(path)); } Status FileSystem::delete_directory(const Path& dir) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(delete_directory_impl(path)); } Status FileSystem::batch_delete(const std::vector& files) { std::vector abs_files; for (auto& file : files) { - abs_files.push_back(absolute_path(file)); + Path abs_file; + RETURN_IF_ERROR(absolute_path(file, abs_file)); + abs_files.push_back(abs_file); } FILESYSTEM_M(batch_delete_impl(abs_files)); } Status FileSystem::exists(const Path& path, bool* res) const { - auto fs_path = absolute_path(path); + Path fs_path; + RETURN_IF_ERROR(absolute_path(path, fs_path)); FILESYSTEM_M(exists_impl(fs_path, res)); } Status FileSystem::file_size(const Path& file, int64_t* file_size) const { - auto path = absolute_path(file); + Path path; + RETURN_IF_ERROR(absolute_path(file, path)); FILESYSTEM_M(file_size_impl(path, file_size)); } Status FileSystem::list(const Path& dir, bool only_file, std::vector* files, bool* exists) { - auto path = absolute_path(dir); + Path path; + RETURN_IF_ERROR(absolute_path(dir, path)); FILESYSTEM_M(list_impl(path, only_file, files, exists)); } Status FileSystem::rename(const Path& orig_name, const Path& new_name) { - auto orig_path = absolute_path(orig_name); - auto new_path = absolute_path(new_name); + Path orig_path; + RETURN_IF_ERROR(absolute_path(orig_name, orig_path)); + Path new_path; + RETURN_IF_ERROR(absolute_path(new_name, new_path)); FILESYSTEM_M(rename_impl(orig_path, new_path)); } diff --git a/be/src/io/fs/file_system.h b/be/src/io/fs/file_system.h index a8ccc8756bb60a..6baf07917d35cb 100644 --- a/be/src/io/fs/file_system.h +++ b/be/src/io/fs/file_system.h @@ -163,7 +163,7 @@ class FileSystem { // FIMXE(plat1ko): The implementation and semantics of this function are not completely // consistent, which is confused. - virtual Path absolute_path(const Path& path) const = 0; + virtual Status absolute_path(const Path& path, Path& abs_path) const = 0; FileSystem(std::string id, FileSystemType type) : _id(std::move(id)), _type(type) {} diff --git a/be/src/io/fs/local_file_system.cpp b/be/src/io/fs/local_file_system.cpp index 0107ed57dc8fb1..9270d919a37519 100644 --- a/be/src/io/fs/local_file_system.cpp +++ b/be/src/io/fs/local_file_system.cpp @@ -471,4 +471,54 @@ Status LocalFileSystem::permission_impl(const Path& file, std::filesystem::perms return Status::OK(); } +Status LocalFileSystem::convert_to_abs_path(const Path& input_path_str, Path& abs_path) { + // valid path include: + // 1. abc/def will return abc/def + // 2. /abc/def will return /abc/def + // 3. file:/abc/def will return /abc/def + // 4. file:///abc/def will return /abc/def + std::string path_str = input_path_str; + size_t slash = path_str.find('/'); + if (slash == 0) { + abs_path = input_path_str; + return Status::OK(); + } + + // Initialize scheme and authority + std::string scheme; + size_t start = 0; + + // Parse URI scheme + size_t colon = path_str.find(':'); + if (colon != std::string::npos && (slash == std::string::npos || colon < slash)) { + // Has a scheme + scheme = path_str.substr(0, colon); + if (scheme != "file") { + return Status::InternalError( + "Only supports `file` type scheme, like 'file:///path', 'file:/path'."); + } + start = colon + 1; + } + + // Parse URI authority, if any + if (path_str.compare(start, 2, "//") == 0 && path_str.length() - start > 2) { + // Has authority + // such as : path_str = "file://authority/abc/def" + // and now : start = 5 + size_t next_slash = path_str.find('/', start + 2); + // now : next_slash = 16 + if (next_slash == std::string::npos) { + return Status::InternalError( + "This input string only has authority, but has no path information"); + } + // We will skit authority + // now : start = 16 + start = next_slash; + } + + // URI path is the rest of the string + abs_path = path_str.substr(start); + return Status::OK(); +} + } // namespace doris::io diff --git a/be/src/io/fs/local_file_system.h b/be/src/io/fs/local_file_system.h index c6295b0bae1d6c..4540df47c16d81 100644 --- a/be/src/io/fs/local_file_system.h +++ b/be/src/io/fs/local_file_system.h @@ -34,6 +34,8 @@ class LocalFileSystem final : public FileSystem { public: ~LocalFileSystem() override; + static Status convert_to_abs_path(const Path& path, Path& abs_path); + /// hard link dest file to src file Status link_file(const Path& src, const Path& dest); @@ -104,7 +106,9 @@ class LocalFileSystem final : public FileSystem { // `LocalFileSystem` always use absolute path as arguments // FIXME(plat1ko): Eliminate this method - Path absolute_path(const Path& path) const override { return path; } + Status absolute_path(const Path& path, Path& abs_path) const override { + return convert_to_abs_path(path, abs_path); + } friend const std::shared_ptr& global_local_filesystem(); }; diff --git a/be/src/io/fs/remote_file_system.cpp b/be/src/io/fs/remote_file_system.cpp index 2b6af2af046afc..fd793f60cdc5d8 100644 --- a/be/src/io/fs/remote_file_system.cpp +++ b/be/src/io/fs/remote_file_system.cpp @@ -29,7 +29,8 @@ namespace doris::io { Status RemoteFileSystem::upload(const Path& local_file, const Path& dest_file) { - auto dest_path = absolute_path(dest_file); + Path dest_path; + RETURN_IF_ERROR(absolute_path(dest_file, dest_path)); FILESYSTEM_M(upload_impl(local_file, dest_path)); } @@ -37,13 +38,16 @@ Status RemoteFileSystem::batch_upload(const std::vector& local_files, const std::vector& remote_files) { std::vector remote_paths; for (auto& path : remote_files) { - remote_paths.push_back(absolute_path(path)); + Path abs_path; + RETURN_IF_ERROR(absolute_path(path, abs_path)); + remote_paths.push_back(abs_path); } FILESYSTEM_M(batch_upload_impl(local_files, remote_paths)); } Status RemoteFileSystem::download(const Path& remote_file, const Path& local) { - auto remote_path = absolute_path(remote_file); + Path remote_path; + RETURN_IF_ERROR(absolute_path(remote_file, remote_path)); FILESYSTEM_M(download_impl(remote_path, local)); } diff --git a/be/src/io/fs/remote_file_system.h b/be/src/io/fs/remote_file_system.h index e9472140ab7b08..de0a1b71519a92 100644 --- a/be/src/io/fs/remote_file_system.h +++ b/be/src/io/fs/remote_file_system.h @@ -64,11 +64,13 @@ class RemoteFileSystem : public FileSystem { virtual Status open_file_internal(const Path& file, FileReaderSPtr* reader, const FileReaderOptions& opts) = 0; - Path absolute_path(const Path& path) const override { + Status absolute_path(const Path& path, Path& abs_path) const override { if (path.is_absolute()) { - return path; + abs_path = path; + } else { + abs_path = _root_path / path; } - return _root_path / path; + return Status::OK(); } Path _root_path; diff --git a/be/src/io/fs/s3_file_system.h b/be/src/io/fs/s3_file_system.h index d1e8b5b6e31a61..f6efa5053324ff 100644 --- a/be/src/io/fs/s3_file_system.h +++ b/be/src/io/fs/s3_file_system.h @@ -113,16 +113,17 @@ class S3FileSystem final : public RemoteFileSystem { const std::vector& remote_files) override; Status download_impl(const Path& remote_file, const Path& local_file) override; - Path absolute_path(const Path& path) const override { + Status absolute_path(const Path& path, Path& abs_path) const override { if (path.string().find("://") != std::string::npos) { // the path is with schema, which means this is a full path like: // s3://bucket/path/to/file.txt // so no need to concat with prefix - return path; + abs_path = path; } else { // path with no schema - return _root_path / path; + abs_path = _prefix / path; } + return Status::OK(); } private: diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 0cb918448c4de8..d4360f7da96f54 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -1547,8 +1547,8 @@ std::vector BaseTablet::get_snapshot_rowset(bool include_stale_ void BaseTablet::calc_consecutive_empty_rowsets( std::vector* empty_rowsets, - const std::vector& candidate_rowsets, int limit) { - int len = candidate_rowsets.size(); + const std::vector& candidate_rowsets, int64_t limit) { + int len = static_cast(candidate_rowsets.size()); for (int i = 0; i < len - 1; ++i) { auto rowset = candidate_rowsets[i]; auto next_rowset = candidate_rowsets[i + 1]; diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index db464dcc6e15d5..7e5ecfe9612ad7 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -263,10 +263,13 @@ class BaseTablet { // Find the first consecutive empty rowsets. output->size() >= limit void calc_consecutive_empty_rowsets(std::vector* empty_rowsets, const std::vector& candidate_rowsets, - int limit); + int64_t limit); // Return the merged schema of all rowsets - virtual TabletSchemaSPtr merged_tablet_schema() const { return _max_version_schema; } + virtual TabletSchemaSPtr merged_tablet_schema() const { + std::shared_lock rlock(_meta_lock); + return _max_version_schema; + } void traverse_rowsets(std::function visitor, bool include_stale = false) { diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index d3bd0f0a3a2436..11249bafb1e3c0 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -35,6 +35,7 @@ #include #include +#include "common/config.h" #include "io/io_common.h" #include "olap/olap_define.h" #include "olap/rowset/rowset_fwd.h" @@ -394,6 +395,8 @@ using ColumnId = uint32_t; using UniqueIdSet = std::set; // Column unique Id -> column id map using UniqueIdToColumnIdMap = std::map; +struct RowsetId; +RowsetId next_rowset_id(); // 8 bit rowset id version // 56 bit, inc number from 1 @@ -412,7 +415,12 @@ struct RowsetId { auto [_, ec] = std::from_chars(rowset_id_str.data(), rowset_id_str.data() + rowset_id_str.length(), high); if (ec != std::errc {}) [[unlikely]] { - LOG(FATAL) << "failed to init rowset id: " << rowset_id_str; + if (config::force_regenerate_rowsetid_on_start_error) { + LOG(WARNING) << "failed to init rowset id: " << rowset_id_str; + high = next_rowset_id().hi; + } else { + LOG(FATAL) << "failed to init rowset id: " << rowset_id_str; + } } init(1, high, 0, 0); } else { diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index a0c5a05636bfa2..b4d0f56288d8d8 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -1040,7 +1040,8 @@ Status StorageEngine::_submit_compaction_task(TabletSharedPtr tablet, if (!tablet->can_do_compaction(tablet->data_dir()->path_hash(), compaction_type)) { LOG(INFO) << "Tablet state has been changed, no need to begin this compaction " "task, tablet_id=" - << tablet->tablet_id() << "tablet_state=" << tablet->tablet_state(); + << tablet->tablet_id() << ", tablet_state=" << tablet->tablet_state(); + _pop_tablet_from_submitted_compaction(tablet, compaction_type); return; } tablet->compaction_stage = CompactionStage::EXECUTING; diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index 435c080c6f006e..f04c765908ca1a 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -60,6 +60,7 @@ #include "vec/data_types/data_type_factory.hpp" namespace doris { +#include "common/compile_check_begin.h" using namespace ErrorCode; namespace { @@ -341,6 +342,14 @@ Status BaseBetaRowsetWriter::_generate_delete_bitmap(int32_t segment_id) { return Status::OK(); } +Status BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) { + RETURN_IF_ERROR(BaseBetaRowsetWriter::init(rowset_writer_context)); + if (_segcompaction_worker) { + _segcompaction_worker->init_mem_tracker(rowset_writer_context); + } + return Status::OK(); +} + Status BetaRowsetWriter::_load_noncompacted_segment(segment_v2::SegmentSharedPtr& segment, int32_t segment_id) { DCHECK(_rowset_meta->is_local()); @@ -467,15 +476,15 @@ Status BetaRowsetWriter::_rename_compacted_segments(int64_t begin, int64_t end) return Status::OK(); } -void BetaRowsetWriter::_clear_statistics_for_deleting_segments_unsafe(uint64_t begin, - uint64_t end) { +void BetaRowsetWriter::_clear_statistics_for_deleting_segments_unsafe(uint32_t begin, + uint32_t end) { VLOG_DEBUG << "_segid_statistics_map clear record segid range from:" << begin << " to:" << end; - for (int i = begin; i <= end; ++i) { + for (uint32_t i = begin; i <= end; ++i) { _segid_statistics_map.erase(i); } } -Status BetaRowsetWriter::_rename_compacted_segment_plain(uint64_t seg_id) { +Status BetaRowsetWriter::_rename_compacted_segment_plain(uint32_t seg_id) { if (seg_id == _num_segcompacted) { ++_num_segcompacted; return Status::OK(); @@ -645,7 +654,7 @@ Status BaseBetaRowsetWriter::add_rowset(RowsetSharedPtr rowset) { _num_rows_written += rowset->num_rows(); _total_data_size += rowset->rowset_meta()->data_disk_size(); _total_index_size += rowset->rowset_meta()->index_disk_size(); - _num_segment += rowset->num_segments(); + _num_segment += static_cast(rowset->num_segments()); // append key_bounds to current rowset RETURN_IF_ERROR(rowset->get_segments_key_bounds(&_segments_encoded_key_bounds)); @@ -1035,7 +1044,7 @@ Status BaseBetaRowsetWriter::add_segment(uint32_t segment_id, const SegmentStati if (segment_id >= _segment_num_rows.size()) { _segment_num_rows.resize(segment_id + 1); } - _segment_num_rows[segid_offset] = segstat.row_num; + _segment_num_rows[segid_offset] = static_cast(segstat.row_num); } VLOG_DEBUG << "_segid_statistics_map add new record. segment_id:" << segment_id << " row_num:" << segstat.row_num << " data_size:" << segstat.data_size @@ -1103,4 +1112,5 @@ Status BetaRowsetWriter::flush_segment_writer_for_segcompaction( return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h index 92ffe1bb092241..760224b055d0c5 100644 --- a/be/src/olap/rowset/beta_rowset_writer.h +++ b/be/src/olap/rowset/beta_rowset_writer.h @@ -273,6 +273,8 @@ class BetaRowsetWriter : public BaseBetaRowsetWriter { Status build(RowsetSharedPtr& rowset) override; + Status init(const RowsetWriterContext& rowset_writer_context) override; + Status add_segment(uint32_t segment_id, const SegmentStatistics& segstat, TabletSchemaSPtr flush_schema) override; @@ -296,9 +298,9 @@ class BetaRowsetWriter : public BaseBetaRowsetWriter { Status _load_noncompacted_segment(segment_v2::SegmentSharedPtr& segment, int32_t segment_id); Status _find_longest_consecutive_small_segment(SegCompactionCandidatesSharedPtr& segments); Status _rename_compacted_segments(int64_t begin, int64_t end); - Status _rename_compacted_segment_plain(uint64_t seg_id); + Status _rename_compacted_segment_plain(uint32_t seg_id); Status _rename_compacted_indices(int64_t begin, int64_t end, uint64_t seg_id); - void _clear_statistics_for_deleting_segments_unsafe(uint64_t begin, uint64_t end); + void _clear_statistics_for_deleting_segments_unsafe(uint32_t begin, uint32_t end); StorageEngine& _engine; diff --git a/be/src/olap/rowset/segcompaction.cpp b/be/src/olap/rowset/segcompaction.cpp index 413e615033628f..89d28e6462af95 100644 --- a/be/src/olap/rowset/segcompaction.cpp +++ b/be/src/olap/rowset/segcompaction.cpp @@ -69,6 +69,19 @@ using namespace ErrorCode; SegcompactionWorker::SegcompactionWorker(BetaRowsetWriter* writer) : _writer(writer) {} +void SegcompactionWorker::init_mem_tracker(const RowsetWriterContext& rowset_writer_context) { + _seg_compact_mem_tracker = MemTrackerLimiter::create_shared( + MemTrackerLimiter::Type::COMPACTION, + fmt::format("segcompaction-txnID_{}-loadID_{}-tabletID_{}-indexID_{}-" + "partitionID_{}-version_{}", + std::to_string(rowset_writer_context.txn_id), + print_id(rowset_writer_context.load_id), + std::to_string(rowset_writer_context.tablet_id), + std::to_string(rowset_writer_context.index_id), + std::to_string(rowset_writer_context.partition_id), + rowset_writer_context.version.to_string())); +} + Status SegcompactionWorker::_get_segcompaction_reader( SegCompactionCandidatesSharedPtr segments, TabletSharedPtr tablet, std::shared_ptr schema, OlapReaderStatistics* stat, diff --git a/be/src/olap/rowset/segcompaction.h b/be/src/olap/rowset/segcompaction.h index f0f8aa6b25787c..973e3fbc5e8f0a 100644 --- a/be/src/olap/rowset/segcompaction.h +++ b/be/src/olap/rowset/segcompaction.h @@ -69,6 +69,8 @@ class SegcompactionWorker { // set the cancel flag, tasks already started will not be cancelled. bool cancel(); + void init_mem_tracker(const RowsetWriterContext& rowset_writer_context); + private: Status _create_segment_writer_for_segcompaction( std::unique_ptr* writer, uint32_t begin, uint32_t end); @@ -95,6 +97,7 @@ class SegcompactionWorker { // for unique key mow table std::unique_ptr _rowid_conversion; DeleteBitmapPtr _converted_delete_bitmap; + std::shared_ptr _seg_compact_mem_tracker = nullptr; // the state is not mutable when 1)actual compaction operation started or 2) cancelled std::atomic _is_compacting_state_mutable = true; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_common.h b/be/src/olap/rowset/segment_v2/inverted_index_common.h new file mode 100644 index 00000000000000..1fdb7df2931de4 --- /dev/null +++ b/be/src/olap/rowset/segment_v2/inverted_index_common.h @@ -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. + +#pragma once + +#include // IWYU pragma: keep + +#include + +#include "common/logging.h" + +namespace lucene::store { +class Directory; +} // namespace lucene::store + +namespace doris::segment_v2 { + +struct DirectoryDeleter { + void operator()(lucene::store::Directory* ptr) const { _CLDECDELETE(ptr); } +}; + +struct ErrorContext { + std::string err_msg; + std::exception_ptr eptr; +}; + +template +concept HasClose = requires(T t) { + { t->close() }; +}; + +template + requires HasClose +void finally_close(PtrType& resource, ErrorContext& error_context) { + if (resource) { + try { + resource->close(); + } catch (CLuceneError& err) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("Error occurred while closing resource: "); + error_context.err_msg.append(err.what()); + LOG(ERROR) << error_context.err_msg; + } catch (...) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("Error occurred while closing resource"); + LOG(ERROR) << error_context.err_msg; + } + } +} + +#if defined(__clang__) +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunused-macros" +#endif + +#define FINALLY_CLOSE(resource) \ + { \ + static_assert(sizeof(error_context) > 0, \ + "error_context must be defined before using FINALLY macro!"); \ + finally_close(resource, error_context); \ + } + +// Return ERROR after finally +#define FINALLY(finally_block) \ + { \ + static_assert(sizeof(error_context) > 0, \ + "error_context must be defined before using FINALLY macro!"); \ + finally_block; \ + if (error_context.eptr) { \ + return Status::Error(error_context.err_msg); \ + } \ + } + +// Re-throw the exception after finally +#define FINALLY_EXCEPTION(finally_block) \ + { \ + static_assert(sizeof(error_context) > 0, \ + "error_context must be defined before using FINALLY macro!"); \ + finally_block; \ + if (error_context.eptr) { \ + std::rethrow_exception(error_context.eptr); \ + } \ + } + +#if defined(__clang__) +#pragma clang diagnostic pop +#endif + +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp index 2d50730daffe8a..bb373be5ee906a 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.cpp @@ -243,10 +243,9 @@ void InvertedIndexFileWriter::copyFile(const char* fileName, lucene::store::Dire Status InvertedIndexFileWriter::write_v1() { int64_t total_size = 0; - std::string err_msg; - lucene::store::Directory* out_dir = nullptr; - std::exception_ptr eptr; + std::unique_ptr out_dir = nullptr; std::unique_ptr output = nullptr; + ErrorContext error_context; for (const auto& entry : _indices_dirs) { const int64_t index_id = entry.first.first; const auto& index_suffix = entry.first.second; @@ -262,7 +261,7 @@ Status InvertedIndexFileWriter::write_v1() { // Create output stream auto result = create_output_stream_v1(index_id, index_suffix); - out_dir = result.first; + out_dir = std::move(result.first); output = std::move(result.second); size_t start = output->getFilePointer(); @@ -275,23 +274,19 @@ Status InvertedIndexFileWriter::write_v1() { total_size += compound_file_size; add_index_info(index_id, index_suffix, compound_file_size); } catch (CLuceneError& err) { - eptr = std::current_exception(); + error_context.eptr = std::current_exception(); auto index_path = InvertedIndexDescriptor::get_index_file_path_v1( _index_path_prefix, index_id, index_suffix); - err_msg = "CLuceneError occur when write_v1 idx file " + index_path + - " error msg: " + err.what(); - } - - // Close and clean up - finalize_output_dir(out_dir); - if (output) { - output->close(); - } - - if (eptr) { - LOG(ERROR) << err_msg; - return Status::Error(err_msg); + error_context.err_msg.append("CLuceneError occur when write_v1 idx file: "); + error_context.err_msg.append(index_path); + error_context.err_msg.append(", error msg: "); + error_context.err_msg.append(err.what()); + LOG(ERROR) << error_context.err_msg; } + FINALLY({ + FINALLY_CLOSE(output); + FINALLY_CLOSE(out_dir); + }) } _total_file_size = total_size; @@ -299,10 +294,9 @@ Status InvertedIndexFileWriter::write_v1() { } Status InvertedIndexFileWriter::write_v2() { - std::string err_msg; - lucene::store::Directory* out_dir = nullptr; + std::unique_ptr out_dir = nullptr; std::unique_ptr compound_file_output = nullptr; - std::exception_ptr eptr; + ErrorContext error_context; try { // Calculate header length and initialize offset int64_t current_offset = headerLength(); @@ -311,7 +305,7 @@ Status InvertedIndexFileWriter::write_v2() { // Create output stream auto result = create_output_stream_v2(); - out_dir = result.first; + out_dir = std::move(result.first); compound_file_output = std::move(result.second); // Write version and number of indices @@ -326,22 +320,18 @@ Status InvertedIndexFileWriter::write_v2() { _total_file_size = compound_file_output->getFilePointer(); _file_info.set_index_size(_total_file_size); } catch (CLuceneError& err) { - eptr = std::current_exception(); + error_context.eptr = std::current_exception(); auto index_path = InvertedIndexDescriptor::get_index_file_path_v2(_index_path_prefix); - err_msg = "CLuceneError occur when close idx file " + index_path + - " error msg: " + err.what(); - } - - // Close and clean up - finalize_output_dir(out_dir); - if (compound_file_output) { - compound_file_output->close(); - } - - if (eptr) { - LOG(ERROR) << err_msg; - return Status::Error(err_msg); + error_context.err_msg.append("CLuceneError occur when close idx file: "); + error_context.err_msg.append(index_path); + error_context.err_msg.append(", error msg: "); + error_context.err_msg.append(err.what()); + LOG(ERROR) << error_context.err_msg; } + FINALLY({ + FINALLY_CLOSE(compound_file_output); + FINALLY_CLOSE(out_dir); + }) return Status::OK(); } @@ -369,13 +359,6 @@ std::vector InvertedIndexFileWriter::prepare_sorted_files( return sorted_files; } -void InvertedIndexFileWriter::finalize_output_dir(lucene::store::Directory* out_dir) { - if (out_dir != nullptr) { - out_dir->close(); - _CLDECDELETE(out_dir) - } -} - void InvertedIndexFileWriter::add_index_info(int64_t index_id, const std::string& index_suffix, int64_t compound_file_size) { InvertedIndexFileInfo_IndexInfo index_info; @@ -424,7 +407,8 @@ std::pair InvertedIndexFileWriter::calculate_header_length( return {header_length, header_file_count}; } -std::pair> +std::pair, + std::unique_ptr> InvertedIndexFileWriter::create_output_stream_v1(int64_t index_id, const std::string& index_suffix) { io::Path cfs_path(InvertedIndexDescriptor::get_index_file_path_v1(_index_path_prefix, index_id, @@ -434,6 +418,7 @@ InvertedIndexFileWriter::create_output_stream_v1(int64_t index_id, auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, idx_path.c_str()); out_dir->set_file_writer_opts(_opts); + std::unique_ptr out_dir_ptr(out_dir); auto* out = out_dir->createOutput(idx_name.c_str()); DBUG_EXECUTE_IF("InvertedIndexFileWriter::write_v1_out_dir_createOutput_nullptr", @@ -443,9 +428,9 @@ InvertedIndexFileWriter::create_output_stream_v1(int64_t index_id, "output is nullptr."; _CLTHROWA(CL_ERR_IO, "Create CompoundDirectory output error"); } - std::unique_ptr output(out); - return {out_dir, std::move(output)}; + + return {std::move(out_dir_ptr), std::move(output)}; } void InvertedIndexFileWriter::write_header_and_data_v1(lucene::store::IndexOutput* output, @@ -483,15 +468,20 @@ void InvertedIndexFileWriter::write_header_and_data_v1(lucene::store::IndexOutpu } } -std::pair> +std::pair, + std::unique_ptr> InvertedIndexFileWriter::create_output_stream_v2() { io::Path index_path {InvertedIndexDescriptor::get_index_file_path_v2(_index_path_prefix)}; + auto* out_dir = DorisFSDirectoryFactory::getDirectory(_fs, index_path.parent_path().c_str()); out_dir->set_file_writer_opts(_opts); + std::unique_ptr out_dir_ptr(out_dir); + DCHECK(_idx_v2_writer != nullptr) << "inverted index file writer v2 is nullptr"; auto compound_file_output = std::unique_ptr( out_dir->createOutputV2(_idx_v2_writer.get())); - return std::make_pair(out_dir, std::move(compound_file_output)); + + return {std::move(out_dir_ptr), std::move(compound_file_output)}; } void InvertedIndexFileWriter::write_version_and_indices_count(lucene::store::IndexOutput* output) { diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h index 3a2fcc1e6acaa7..ba42ffdceb1475 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_writer.h @@ -29,6 +29,7 @@ #include "io/fs/file_system.h" #include "io/fs/file_writer.h" #include "io/fs/local_file_system.h" +#include "olap/rowset/segment_v2/inverted_index_common.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" #include "runtime/exec_env.h" @@ -105,21 +106,22 @@ class InvertedIndexFileWriter { void sort_files(std::vector& file_infos); void copyFile(const char* fileName, lucene::store::Directory* dir, lucene::store::IndexOutput* output, uint8_t* buffer, int64_t bufferLength); - void finalize_output_dir(lucene::store::Directory* out_dir); void add_index_info(int64_t index_id, const std::string& index_suffix, int64_t compound_file_size); int64_t headerLength(); // Helper functions specific to write_v1 std::pair calculate_header_length(const std::vector& sorted_files, lucene::store::Directory* directory); - std::pair> + virtual std::pair, + std::unique_ptr> create_output_stream_v1(int64_t index_id, const std::string& index_suffix); virtual void write_header_and_data_v1(lucene::store::IndexOutput* output, const std::vector& sorted_files, lucene::store::Directory* directory, int64_t header_length, int32_t header_file_count); // Helper functions specific to write_v2 - std::pair> + virtual std::pair, + std::unique_ptr> create_output_stream_v2(); void write_version_and_indices_count(lucene::store::IndexOutput* output); struct FileMetadata { 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 a4f3ca55dd11c0..86a8f89e4c94e4 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp @@ -51,6 +51,7 @@ #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/inverted_index/analyzer/analyzer.h" #include "olap/rowset/segment_v2/inverted_index/char_filter/char_filter_factory.h" +#include "olap/rowset/segment_v2/inverted_index_common.h" #include "olap/rowset/segment_v2/inverted_index_desc.h" #include "olap/rowset/segment_v2/inverted_index_file_writer.h" #include "olap/rowset/segment_v2/inverted_index_fs_directory.h" @@ -63,11 +64,6 @@ #include "util/slice.h" #include "util/string_util.h" -#define FINALLY_CLOSE_OUTPUT(x) \ - try { \ - if (x != nullptr) x->close(); \ - } catch (...) { \ - } namespace doris::segment_v2 { const int32_t MAX_FIELD_LEN = 0x7FFFFFFFL; const int32_t MERGE_FACTOR = 100000000; @@ -138,13 +134,6 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { } } - void close() { - if (_index_writer) { - _index_writer->close(); - _index_writer.reset(); - } - } - void close_on_error() override { try { DBUG_EXECUTE_IF("InvertedIndexColumnWriter::close_on_error_throw_exception", @@ -618,7 +607,6 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { buf.resize(size); _null_bitmap.write(reinterpret_cast(buf.data()), false); null_bitmap_out->writeBytes(buf.data(), size); - null_bitmap_out->close(); } } @@ -628,6 +616,7 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { std::unique_ptr data_out = nullptr; std::unique_ptr index_out = nullptr; std::unique_ptr meta_out = nullptr; + ErrorContext error_context; try { // write bkd file if constexpr (field_is_numeric_type(field_type)) { @@ -656,16 +645,11 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { << "Inverted index writer create output error occurred: nullptr"; _CLTHROWA(CL_ERR_IO, "Create output error with nullptr"); } - meta_out->close(); - data_out->close(); - index_out->close(); - _dir->close(); } else if constexpr (field_is_slice_type(field_type)) { null_bitmap_out = std::unique_ptr< lucene::store::IndexOutput>(_dir->createOutput( InvertedIndexDescriptor::get_temporary_null_bitmap_file_name())); write_null_bitmap(null_bitmap_out.get()); - close(); DBUG_EXECUTE_IF( "InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close", { _CLTHROWA(CL_ERR_IO, @@ -673,19 +657,24 @@ class InvertedIndexColumnWriterImpl : public InvertedIndexColumnWriter { }); } } catch (CLuceneError& e) { - FINALLY_CLOSE_OUTPUT(null_bitmap_out) - FINALLY_CLOSE_OUTPUT(meta_out) - FINALLY_CLOSE_OUTPUT(data_out) - FINALLY_CLOSE_OUTPUT(index_out) + error_context.eptr = std::current_exception(); + error_context.err_msg.append("Inverted index writer finish error occurred: "); + error_context.err_msg.append(e.what()); + LOG(ERROR) << error_context.err_msg; + } + FINALLY({ + FINALLY_CLOSE(null_bitmap_out); + FINALLY_CLOSE(meta_out); + FINALLY_CLOSE(data_out); + FINALLY_CLOSE(index_out); if constexpr (field_is_numeric_type(field_type)) { - FINALLY_CLOSE_OUTPUT(_dir) + FINALLY_CLOSE(_dir); } else if constexpr (field_is_slice_type(field_type)) { - FINALLY_CLOSE_OUTPUT(_index_writer); + FINALLY_CLOSE(_index_writer); + // After closing the _index_writer, it needs to be reset to null to prevent issues of not closing it or closing it multiple times. + _index_writer.reset(); } - LOG(WARNING) << "Inverted index writer finish error occurred: " << e.what(); - return Status::Error( - "Inverted index writer finish error occurred:{}", e.what()); - } + }) return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index 0db06c2de74732..2a61eb3f0fe8f7 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -519,6 +519,40 @@ Status SegmentWriter::probe_key_for_mow( return Status::OK(); } +Status SegmentWriter::partial_update_preconditions_check(size_t row_pos) { + if (!_is_mow()) { + auto msg = fmt::format( + "Can only do partial update on merge-on-write unique table, but found: " + "keys_type={}, _opts.enable_unique_key_merge_on_write={}, tablet_id={}", + _tablet_schema->keys_type(), _opts.enable_unique_key_merge_on_write, + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + if (_opts.rowset_ctx->partial_update_info == nullptr) { + auto msg = + fmt::format("partial_update_info should not be nullptr, please check, tablet_id={}", + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + if (!_opts.rowset_ctx->partial_update_info->is_partial_update) { + auto msg = fmt::format( + "in fixed partial update code, but is_partial_update=false, please check, " + "tablet_id={}", + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + if (row_pos != 0) { + auto msg = fmt::format("row_pos should be 0, but found {}, tablet_id={}", row_pos, + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + return Status::OK(); +} + // for partial update, we should do following steps to fill content of block: // 1. set block data to data convertor, and get all key_column's converted slice // 2. get pk of input block, and read missing columns @@ -536,10 +570,7 @@ Status SegmentWriter::append_block_with_partial_content(const vectorized::Block* block->columns(), _tablet_schema->num_key_columns(), _tablet_schema->num_columns())); } - DCHECK(_is_mow()); - - DCHECK(_opts.rowset_ctx->partial_update_info); - DCHECK(row_pos == 0); + RETURN_IF_ERROR(partial_update_preconditions_check(row_pos)); // find missing column cids const auto& missing_cids = _opts.rowset_ctx->partial_update_info->missing_cids; diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h index 4b157092221d49..02ae962a16a942 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.h +++ b/be/src/olap/rowset/segment_v2/segment_writer.h @@ -102,6 +102,7 @@ class SegmentWriter { bool& has_default_or_nullable, std::vector& use_default_or_null_flag, PartialUpdateStats& stats); + Status partial_update_preconditions_check(size_t row_pos); Status append_block_with_partial_content(const vectorized::Block* block, size_t row_pos, size_t num_rows); Status append_block_with_variant_subcolumns(vectorized::Block& data); diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp index 857f6b2f3c7f0b..cdbdcab47912d4 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp @@ -390,6 +390,39 @@ Status VerticalSegmentWriter::_probe_key_for_mow( return Status::OK(); } +Status VerticalSegmentWriter::_partial_update_preconditions_check(size_t row_pos) { + if (!_is_mow()) { + auto msg = fmt::format( + "Can only do partial update on merge-on-write unique table, but found: " + "keys_type={}, _opts.enable_unique_key_merge_on_write={}, tablet_id={}", + _tablet_schema->keys_type(), _opts.enable_unique_key_merge_on_write, + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + if (_opts.rowset_ctx->partial_update_info == nullptr) { + auto msg = + fmt::format("partial_update_info should not be nullptr, please check, tablet_id={}", + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + if (!_opts.rowset_ctx->partial_update_info->is_partial_update) { + auto msg = fmt::format( + "in partial update code, but is_partial_update=false, please check, tablet_id={}", + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + if (row_pos != 0) { + auto msg = fmt::format("row_pos should be 0, but found {}, tablet_id={}", row_pos, + _tablet->tablet_id()); + DCHECK(false) << msg; + return Status::InternalError(msg); + } + return Status::OK(); +} + // for partial update, we should do following steps to fill content of block: // 1. set block data to data convertor, and get all key_column's converted slice // 2. get pk of input block, and read missing columns @@ -399,9 +432,7 @@ Status VerticalSegmentWriter::_probe_key_for_mow( // 3. set columns to data convertor and then write all columns Status VerticalSegmentWriter::_append_block_with_partial_content(RowsInBlock& data, vectorized::Block& full_block) { - DCHECK(_is_mow()); - DCHECK(_opts.rowset_ctx->partial_update_info != nullptr); - DCHECK(data.row_pos == 0); + RETURN_IF_ERROR(_partial_update_preconditions_check(data.row_pos)); // create full block and fill with input columns full_block = _tablet_schema->create_block(); diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.h b/be/src/olap/rowset/segment_v2/vertical_segment_writer.h index 0e04b49b343709..a327873bbce356 100644 --- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.h +++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.h @@ -173,6 +173,7 @@ class VerticalSegmentWriter { bool& has_default_or_nullable, std::vector& use_default_or_null_flag, PartialUpdateStats& stats); + Status _partial_update_preconditions_check(size_t row_pos); Status _append_block_with_partial_content(RowsInBlock& data, vectorized::Block& full_block); Status _append_block_with_variant_subcolumns(RowsInBlock& data); Status _generate_key_index( diff --git a/be/src/olap/rowset/unique_rowset_id_generator.cpp b/be/src/olap/rowset/unique_rowset_id_generator.cpp index 0ac7f63837a099..49e07e5835957a 100644 --- a/be/src/olap/rowset/unique_rowset_id_generator.cpp +++ b/be/src/olap/rowset/unique_rowset_id_generator.cpp @@ -17,8 +17,17 @@ #include "olap/rowset/unique_rowset_id_generator.h" +#include + +#include "olap/storage_engine.h" +#include "runtime/exec_env.h" + namespace doris { +RowsetId next_rowset_id() { + return ExecEnv::GetInstance()->storage_engine().next_rowset_id(); +} + UniqueRowsetIdGenerator::UniqueRowsetIdGenerator(const UniqueId& backend_uid) : _backend_uid(backend_uid), _inc_id(1) {} diff --git a/be/src/olap/rowset_builder.cpp b/be/src/olap/rowset_builder.cpp index 1bf00ec889550f..4a95febbbb86fa 100644 --- a/be/src/olap/rowset_builder.cpp +++ b/be/src/olap/rowset_builder.cpp @@ -148,8 +148,13 @@ Status BaseRowsetBuilder::init_mow_context(std::shared_ptr& mow_cont } Status RowsetBuilder::check_tablet_version_count() { - if (!_tablet->exceed_version_limit(config::max_tablet_version_num - 100) || - GlobalMemoryArbitrator::is_exceed_soft_mem_limit(GB_EXCHANGE_BYTE)) { + bool injection = false; + DBUG_EXECUTE_IF("RowsetBuilder.check_tablet_version_count.too_many_version", + { injection = true; }); + if (injection) { + // do not return if injection + } else if (!_tablet->exceed_version_limit(config::max_tablet_version_num - 100) || + GlobalMemoryArbitrator::is_exceed_soft_mem_limit(GB_EXCHANGE_BYTE)) { return Status::OK(); } //trigger compaction diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 14d5070018e8b5..4349bfa9b95384 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -2507,10 +2507,10 @@ void Tablet::set_skip_compaction(bool skip, CompactionType compaction_type, int6 bool Tablet::should_skip_compaction(CompactionType compaction_type, int64_t now) { if (compaction_type == CompactionType::CUMULATIVE_COMPACTION && _skip_cumu_compaction && - now < _skip_cumu_compaction_ts + 120) { + now < _skip_cumu_compaction_ts + config::skip_tablet_compaction_second) { return true; } else if (compaction_type == CompactionType::BASE_COMPACTION && _skip_base_compaction && - now < _skip_base_compaction_ts + 120) { + now < _skip_base_compaction_ts + config::skip_tablet_compaction_second) { return true; } return false; diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 5a5f98c969e332..0bd89cc96ac7f9 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1180,14 +1180,14 @@ bool TabletManager::_move_tablet_to_trash(const TabletSharedPtr& tablet) { if (tablet_in_not_shutdown->tablet_path() != tablet->tablet_path()) { LOG(INFO) << "tablet path not eq shutdown tablet path, move it to trash, tablet_id=" << tablet_in_not_shutdown->tablet_id() - << " mem manager tablet path=" << tablet_in_not_shutdown->tablet_path() - << " shutdown tablet path=" << tablet->tablet_path(); + << ", mem manager tablet path=" << tablet_in_not_shutdown->tablet_path() + << ", shutdown tablet path=" << tablet->tablet_path(); return tablet->data_dir()->move_to_trash(tablet->tablet_path()); } else { LOG(INFO) << "tablet path eq shutdown tablet path, not move to trash, tablet_id=" << tablet_in_not_shutdown->tablet_id() - << " mem manager tablet path=" << tablet_in_not_shutdown->tablet_path() - << " shutdown tablet path=" << tablet->tablet_path(); + << ", mem manager tablet path=" << tablet_in_not_shutdown->tablet_path() + << ", shutdown tablet path=" << tablet->tablet_path(); return true; } } @@ -1292,7 +1292,7 @@ Status TabletManager::register_transition_tablet(int64_t tablet_id, std::string // not found shard.tablets_under_transition[tablet_id] = std::make_tuple(reason, thread_id, 1); LOG(INFO) << "add tablet_id= " << tablet_id << " to map, reason=" << reason - << " lock times=1 thread_id_in_map=" << thread_id; + << ", lock times=1, thread_id_in_map=" << thread_id; return Status::OK(); } else { // found @@ -1300,15 +1300,15 @@ Status TabletManager::register_transition_tablet(int64_t tablet_id, std::string if (thread_id != thread_id_in_map) { // other thread, failed LOG(INFO) << "tablet_id = " << tablet_id << " is doing " << r - << " thread_id_in_map=" << thread_id_in_map << " , add reason=" << reason - << " thread_id=" << thread_id; + << ", thread_id_in_map=" << thread_id_in_map << " , add reason=" << reason + << ", thread_id=" << thread_id; return Status::InternalError("{} failed try later, tablet_id={}", reason, tablet_id); } // add lock times ++lock_times; LOG(INFO) << "add tablet_id= " << tablet_id << " to map, reason=" << reason - << " lock times=" << lock_times << " thread_id_in_map=" << thread_id_in_map; + << ", lock times=" << lock_times << ", thread_id_in_map=" << thread_id_in_map; return Status::OK(); } } @@ -1332,10 +1332,10 @@ void TabletManager::unregister_transition_tablet(int64_t tablet_id, std::string --lock_times; if (lock_times != 0) { LOG(INFO) << "erase tablet_id= " << tablet_id << " from map, reason=" << reason - << " left=" << lock_times << " thread_id_in_map=" << thread_id_in_map; + << ", left=" << lock_times << ", thread_id_in_map=" << thread_id_in_map; } else { LOG(INFO) << "erase tablet_id= " << tablet_id << " from map, reason=" << reason - << " thread_id_in_map=" << thread_id_in_map; + << ", thread_id_in_map=" << thread_id_in_map; shard.tablets_under_transition.erase(tablet_id); } } diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 0ebbaf0a65b307..a8dd0e56273e46 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -57,6 +57,7 @@ using std::unordered_map; using std::vector; namespace doris { +#include "common/compile_check_begin.h" using namespace ErrorCode; TabletMetaSharedPtr TabletMeta::create( @@ -106,7 +107,7 @@ TabletMeta::TabletMeta() _delete_bitmap(new DeleteBitmap(_tablet_id)) {} TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id, - int64_t replica_id, int32_t schema_hash, uint64_t shard_id, + int64_t replica_id, int32_t schema_hash, int32_t shard_id, const TTabletSchema& tablet_schema, uint32_t next_unique_id, const std::unordered_map& col_ordinal_to_unique_id, TabletUid tablet_uid, TTabletType::type tabletType, @@ -567,7 +568,8 @@ void TabletMeta::serialize(string* meta_binary) { Status TabletMeta::deserialize(std::string_view meta_binary) { TabletMetaPB tablet_meta_pb; - bool parsed = tablet_meta_pb.ParseFromArray(meta_binary.data(), meta_binary.size()); + bool parsed = tablet_meta_pb.ParseFromArray(meta_binary.data(), + static_cast(meta_binary.size())); if (!parsed) { return Status::Error("parse tablet meta failed"); } @@ -660,11 +662,11 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { int seg_maps_size = tablet_meta_pb.delete_bitmap().segment_delete_bitmaps_size(); CHECK(rst_ids_size == seg_ids_size && seg_ids_size == seg_maps_size && seg_maps_size == versions_size); - for (size_t i = 0; i < rst_ids_size; ++i) { + for (int i = 0; i < rst_ids_size; ++i) { RowsetId rst_id; rst_id.init(tablet_meta_pb.delete_bitmap().rowset_ids(i)); auto seg_id = tablet_meta_pb.delete_bitmap().segment_ids(i); - uint32_t ver = tablet_meta_pb.delete_bitmap().versions(i); + auto ver = tablet_meta_pb.delete_bitmap().versions(i); auto bitmap = tablet_meta_pb.delete_bitmap().segment_delete_bitmaps(i).data(); delete_bitmap().delete_bitmap[{rst_id, seg_id, ver}] = roaring::Roaring::read(bitmap); } @@ -1308,4 +1310,5 @@ std::string tablet_state_name(TabletState state) { } } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index 3a5999c6124684..9f47ef4913ce40 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -50,6 +50,7 @@ #include "util/uid_util.h" namespace json2pb { +#include "common/compile_check_begin.h" struct Pb2JsonOptions; } // namespace json2pb @@ -99,7 +100,7 @@ class TabletMeta { TabletMeta(); TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id, int64_t replica_id, - int32_t schema_hash, uint64_t shard_id, const TTabletSchema& tablet_schema, + int32_t schema_hash, int32_t shard_id, const TTabletSchema& tablet_schema, uint32_t next_unique_id, const std::unordered_map& col_ordinal_to_unique_id, TabletUid tablet_uid, TTabletType::type tabletType, @@ -155,7 +156,7 @@ class TabletMeta { int64_t replica_id() const; void set_replica_id(int64_t replica_id) { _replica_id = replica_id; } int32_t schema_hash() const; - int16_t shard_id() const; + int32_t shard_id() const; void set_shard_id(int32_t shard_id); int64_t creation_time() const; void set_creation_time(int64_t creation_time); @@ -614,7 +615,7 @@ inline int32_t TabletMeta::schema_hash() const { return _schema_hash; } -inline int16_t TabletMeta::shard_id() const { +inline int32_t TabletMeta::shard_id() const { return _shard_id; } @@ -778,4 +779,5 @@ std::string tablet_state_name(TabletState state); bool operator==(const TabletMeta& a, const TabletMeta& b); bool operator!=(const TabletMeta& a, const TabletMeta& b); +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/olap/tablet_meta_manager.cpp b/be/src/olap/tablet_meta_manager.cpp index 6f27dd4db4e672..7c08d7856200f9 100644 --- a/be/src/olap/tablet_meta_manager.cpp +++ b/be/src/olap/tablet_meta_manager.cpp @@ -291,8 +291,7 @@ Status TabletMetaManager::remove_old_version_delete_bitmap(DataDir* store, TTabl return true; }; LOG(INFO) << "remove old version delete bitmap, tablet_id: " << tablet_id - << " version: " << version << " removed keys size: " << remove_keys.size(); - ; + << " version: " << version << ", removed keys size: " << remove_keys.size(); RETURN_IF_ERROR(meta->iterate(META_COLUMN_FAMILY_INDEX, begin_key, get_remove_keys_func)); return meta->remove(META_COLUMN_FAMILY_INDEX, remove_keys); } diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index bea1d3b1a91e89..fa8d9b8248e3f4 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -44,6 +44,7 @@ #include "gutil/strings/split.h" #include "gutil/strings/strip.h" #include "http/http_client.h" +#include "http/utils.h" #include "io/fs/file_system.h" #include "io/fs/local_file_system.h" #include "io/fs/path.h" @@ -399,28 +400,62 @@ Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, .error(st); } }}; - std::string remote_url_prefix; + + std::string remote_dir; { std::stringstream ss; if (snapshot_path->back() == '/') { - ss << "http://" << get_host_port(src.host, src.http_port) << HTTP_REQUEST_PREFIX - << HTTP_REQUEST_TOKEN_PARAM << token << HTTP_REQUEST_FILE_PARAM << *snapshot_path - << _clone_req.tablet_id << "/" << _clone_req.schema_hash << "/"; + ss << *snapshot_path << _clone_req.tablet_id << "/" << _clone_req.schema_hash + << "/"; } else { - ss << "http://" << get_host_port(src.host, src.http_port) << HTTP_REQUEST_PREFIX - << HTTP_REQUEST_TOKEN_PARAM << token << HTTP_REQUEST_FILE_PARAM << *snapshot_path - << "/" << _clone_req.tablet_id << "/" << _clone_req.schema_hash << "/"; + ss << *snapshot_path << "/" << _clone_req.tablet_id << "/" << _clone_req.schema_hash + << "/"; } - remote_url_prefix = ss.str(); + remote_dir = ss.str(); } - status = _download_files(&data_dir, remote_url_prefix, local_data_path); - if (!status.ok()) [[unlikely]] { - LOG_WARNING("failed to download snapshot from remote BE") - .tag("url", mask_token(remote_url_prefix)) - .error(status); - continue; // Try another BE + std::string address = get_host_port(src.host, src.http_port); + if (config::enable_batch_download && is_support_batch_download(address).ok()) { + // download files via batch api. + LOG_INFO("remote BE supports batch download, use batch file download") + .tag("address", address) + .tag("remote_dir", remote_dir); + status = _batch_download_files(&data_dir, address, remote_dir, local_data_path); + if (!status.ok()) [[unlikely]] { + LOG_WARNING("failed to download snapshot from remote BE in batch") + .tag("address", address) + .tag("remote_dir", remote_dir) + .error(status); + continue; // Try another BE + } + } else { + if (config::enable_batch_download) { + LOG_INFO("remote BE does not support batch download, use single file download") + .tag("address", address) + .tag("remote_dir", remote_dir); + } else { + LOG_INFO("batch download is disabled, use single file download") + .tag("address", address) + .tag("remote_dir", remote_dir); + } + + std::string remote_url_prefix; + { + std::stringstream ss; + ss << "http://" << address << HTTP_REQUEST_PREFIX << HTTP_REQUEST_TOKEN_PARAM + << token << HTTP_REQUEST_FILE_PARAM << remote_dir; + remote_url_prefix = ss.str(); + } + + status = _download_files(&data_dir, remote_url_prefix, local_data_path); + if (!status.ok()) [[unlikely]] { + LOG_WARNING("failed to download snapshot from remote BE") + .tag("url", mask_token(remote_url_prefix)) + .error(status); + continue; // Try another BE + } } + // No need to try again with another BE _pending_rs_guards = DORIS_TRY(_engine.snapshot_mgr()->convert_rowset_ids( local_data_path, _clone_req.tablet_id, _clone_req.replica_id, _clone_req.table_id, @@ -514,7 +549,7 @@ Status EngineCloneTask::_download_files(DataDir* data_dir, const std::string& re // If the header file is not exist, the table couldn't loaded by olap engine. // Avoid of data is not complete, we copy the header file at last. // The header file's name is end of .hdr. - for (int i = 0; i < file_name_list.size() - 1; ++i) { + for (int i = 0; i + 1 < file_name_list.size(); ++i) { if (file_name_list[i].ends_with(".hdr")) { std::swap(file_name_list[i], file_name_list[file_name_list.size() - 1]); break; @@ -593,13 +628,91 @@ Status EngineCloneTask::_download_files(DataDir* data_dir, const std::string& re } _copy_size = (int64_t)total_file_size; _copy_time_ms = (int64_t)total_time_ms; - LOG(INFO) << "succeed to copy tablet " << _signature << ", total file size: " << total_file_size - << " B" - << ", cost: " << total_time_ms << " ms" + LOG(INFO) << "succeed to copy tablet " << _signature + << ", total files: " << file_name_list.size() + << ", total file size: " << total_file_size << " B, cost: " << total_time_ms << " ms" << ", rate: " << copy_rate << " MB/s"; return Status::OK(); } +Status EngineCloneTask::_batch_download_files(DataDir* data_dir, const std::string& address, + const std::string& remote_dir, + const std::string& local_dir) { + constexpr size_t BATCH_FILE_SIZE = 64 << 20; // 64MB + constexpr size_t BATCH_FILE_NUM = 64; + + // Check local path exist, if exist, remove it, then create the dir + // local_file_full_path = tabletid/clone, for a specific tablet, there should be only one folder + // if this folder exists, then should remove it + // for example, BE clone from BE 1 to download file 1 with version (2,2), but clone from BE 1 failed + // then it will try to clone from BE 2, but it will find the file 1 already exist, but file 1 with same + // name may have different versions. + RETURN_IF_ERROR(io::global_local_filesystem()->delete_directory(local_dir)); + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(local_dir)); + + const std::string& token = _cluster_info->token; + std::vector> file_info_list; + RETURN_IF_ERROR(list_remote_files_v2(address, token, remote_dir, &file_info_list)); + + // If the header file is not exist, the table couldn't loaded by olap engine. + // Avoid of data is not complete, we copy the header file at last. + // The header file's name is end of .hdr. + for (int i = 0; i + 1 < file_info_list.size(); ++i) { + if (file_info_list[i].first.ends_with(".hdr")) { + std::swap(file_info_list[i], file_info_list[file_info_list.size() - 1]); + break; + } + } + + MonotonicStopWatch watch; + watch.start(); + + size_t total_file_size = 0; + size_t total_files = file_info_list.size(); + std::vector> batch_files; + for (size_t i = 0; i < total_files;) { + size_t batch_file_size = 0; + for (size_t j = i; j < total_files; j++) { + // Split batchs by file number and file size, + if (BATCH_FILE_NUM <= batch_files.size() || BATCH_FILE_SIZE <= batch_file_size || + // ... or separate the last .hdr file into a single batch. + (j + 1 == total_files && !batch_files.empty())) { + break; + } + batch_files.push_back(file_info_list[j]); + batch_file_size += file_info_list[j].second; + } + + // check disk capacity + if (data_dir->reach_capacity_limit(batch_file_size)) { + return Status::Error( + "reach the capacity limit of path {}, file_size={}", data_dir->path(), + batch_file_size); + } + + RETURN_IF_ERROR(download_files_v2(address, token, remote_dir, local_dir, batch_files)); + + total_file_size += batch_file_size; + i += batch_files.size(); + batch_files.clear(); + } + + uint64_t total_time_ms = watch.elapsed_time() / 1000 / 1000; + total_time_ms = total_time_ms > 0 ? total_time_ms : 0; + double copy_rate = 0.0; + if (total_time_ms > 0) { + copy_rate = total_file_size / ((double)total_time_ms) / 1000; + } + _copy_size = (int64_t)total_file_size; + _copy_time_ms = (int64_t)total_time_ms; + LOG(INFO) << "succeed to copy tablet " << _signature + << ", total files: " << file_info_list.size() + << ", total file size: " << total_file_size << " B, cost: " << total_time_ms << " ms" + << ", rate: " << copy_rate << " MB/s"; + + return Status::OK(); +} + /// This method will only be called if tablet already exist in this BE when doing clone. /// This method will do the following things: /// 1. Link all files from CLONE dir to tablet dir if file does not exist in tablet dir diff --git a/be/src/olap/task/engine_clone_task.h b/be/src/olap/task/engine_clone_task.h index a11d4c742f4bcc..e2ced28f03c88d 100644 --- a/be/src/olap/task/engine_clone_task.h +++ b/be/src/olap/task/engine_clone_task.h @@ -79,6 +79,9 @@ class EngineCloneTask final : public EngineTask { Status _download_files(DataDir* data_dir, const std::string& remote_url_prefix, const std::string& local_path); + Status _batch_download_files(DataDir* data_dir, const std::string& endpoint, + const std::string& remote_dir, const std::string& local_dir); + Status _make_snapshot(const std::string& ip, int port, TTableId tablet_id, TSchemaHash schema_hash, int timeout_s, const std::vector& missing_versions, std::string* snapshot_path, diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp index dae4c6be814d5a..75e589f3b97728 100644 --- a/be/src/olap/task/engine_publish_version_task.cpp +++ b/be/src/olap/task/engine_publish_version_task.cpp @@ -82,8 +82,10 @@ EnginePublishVersionTask::EnginePublishVersionTask( _succ_tablets(succ_tablets), _discontinuous_version_tablets(discontinuous_version_tablets), _table_id_to_tablet_id_to_num_delta_rows(table_id_to_tablet_id_to_num_delta_rows) { - _mem_tracker = MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::OTHER, - "TabletPublishTxnTask"); + _mem_tracker = MemTrackerLimiter::create_shared( + MemTrackerLimiter::Type::OTHER, + fmt::format("EnginePublishVersionTask-transactionID_{}", + std::to_string(_publish_version_req.transaction_id))); } void EnginePublishVersionTask::add_error_tablet_id(int64_t tablet_id) { @@ -381,8 +383,11 @@ TabletPublishTxnTask::TabletPublishTxnTask(StorageEngine& engine, _transaction_id(transaction_id), _version(version), _tablet_info(tablet_info), - _mem_tracker(MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::OTHER, - "TabletPublishTxnTask")) { + _mem_tracker(MemTrackerLimiter::create_shared( + MemTrackerLimiter::Type::OTHER, + fmt::format("TabletPublishTxnTask-partitionID_{}-transactionID_{}-version_{}", + std::to_string(partition_id), std::to_string(transaction_id), + version.to_string()))) { _stats.submit_time_us = MonotonicMicros(); } diff --git a/be/src/pipeline/exec/schema_scan_operator.cpp b/be/src/pipeline/exec/schema_scan_operator.cpp index 006ecf8ad82e84..ddc2821cac14a1 100644 --- a/be/src/pipeline/exec/schema_scan_operator.cpp +++ b/be/src/pipeline/exec/schema_scan_operator.cpp @@ -48,7 +48,7 @@ Status SchemaScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { // new one scanner _schema_scanner = SchemaScanner::create(schema_table->schema_table_type()); - _schema_scanner->set_dependency(_data_dependency, _finish_dependency); + _schema_scanner->set_dependency(_data_dependency); if (nullptr == _schema_scanner) { return Status::InternalError("schema scanner get nullptr pointer."); } @@ -266,9 +266,6 @@ Status SchemaScanOperatorX::get_block(RuntimeState* state, vectorized::Block* bl } while (block->rows() == 0 && !*eos); local_state.reached_limit(block, eos); - if (*eos) { - local_state._finish_dependency->set_always_ready(); - } return Status::OK(); } diff --git a/be/src/pipeline/exec/schema_scan_operator.h b/be/src/pipeline/exec/schema_scan_operator.h index 03cf422fbc52e6..c8ddf885e98a0f 100644 --- a/be/src/pipeline/exec/schema_scan_operator.h +++ b/be/src/pipeline/exec/schema_scan_operator.h @@ -36,9 +36,6 @@ class SchemaScanLocalState final : public PipelineXLocalState<> { SchemaScanLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState<>(state, parent) { - _finish_dependency = - std::make_shared(parent->operator_id(), parent->node_id(), - parent->get_name() + "_FINISH_DEPENDENCY", true); _data_dependency = std::make_shared(parent->operator_id(), parent->node_id(), parent->get_name() + "_DEPENDENCY", true); } @@ -48,7 +45,6 @@ class SchemaScanLocalState final : public PipelineXLocalState<> { Status open(RuntimeState* state) override; - Dependency* finishdependency() override { return _finish_dependency.get(); } std::vector dependencies() const override { return {_data_dependency.get()}; } private: @@ -57,7 +53,6 @@ class SchemaScanLocalState final : public PipelineXLocalState<> { SchemaScannerParam _scanner_param; std::unique_ptr _schema_scanner; - std::shared_ptr _finish_dependency; std::shared_ptr _data_dependency; }; diff --git a/be/src/pipeline/local_exchange/local_exchanger.h b/be/src/pipeline/local_exchange/local_exchanger.h index af95e5348c8c4a..274e7b404aa6e2 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.h +++ b/be/src/pipeline/local_exchange/local_exchanger.h @@ -109,7 +109,11 @@ struct BlockQueue { : eos(other.eos.load()), data_queue(std::move(other.data_queue)) {} inline bool enqueue(BlockType const& item) { if (!eos) { - data_queue.enqueue(item); + if (!data_queue.enqueue(item)) [[unlikely]] { + throw Exception(ErrorCode::INTERNAL_ERROR, + "Exception occurs in data queue [size = {}] of local exchange.", + data_queue.size_approx()); + } return true; } return false; @@ -117,7 +121,11 @@ struct BlockQueue { inline bool enqueue(BlockType&& item) { if (!eos) { - data_queue.enqueue(std::move(item)); + if (!data_queue.enqueue(std::move(item))) [[unlikely]] { + throw Exception(ErrorCode::INTERNAL_ERROR, + "Exception occurs in data queue [size = {}] of local exchange.", + data_queue.size_approx()); + } return true; } return false; @@ -185,6 +193,8 @@ struct BlockWrapper { shared_state->exchanger->_free_block_limit * shared_state->exchanger->_num_sources) { data_block.clear_column_data(); + // Free blocks is used to improve memory efficiency. Failure during pushing back + // free block will not incur any bad result so just ignore the return value. shared_state->exchanger->_free_blocks.enqueue(std::move(data_block)); } } diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h index 619848110d4a33..061a62ea99b330 100644 --- a/be/src/pipeline/pipeline.h +++ b/be/src/pipeline/pipeline.h @@ -73,6 +73,14 @@ class Pipeline : public std::enable_shared_from_this { return idx == ExchangeType::HASH_SHUFFLE || idx == ExchangeType::BUCKET_HASH_SHUFFLE; } + // For HASH_SHUFFLE, BUCKET_HASH_SHUFFLE, and ADAPTIVE_PASSTHROUGH, + // data is processed and shuffled on the sink. + // Compared to PASSTHROUGH, this is a relatively heavy operation. + static bool heavy_operations_on_the_sink(ExchangeType idx) { + return idx == ExchangeType::HASH_SHUFFLE || idx == ExchangeType::BUCKET_HASH_SHUFFLE || + idx == ExchangeType::ADAPTIVE_PASSTHROUGH; + } + bool need_to_local_exchange(const DataDistribution target_data_distribution, const int idx) const; void init_data_distribution() { diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 01c14f1ddb3a61..48f2d96597c9cb 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -807,7 +807,7 @@ Status PipelineFragmentContext::_add_local_exchange_impl( } case ExchangeType::ADAPTIVE_PASSTHROUGH: shared_state->exchanger = AdaptivePassthroughExchanger::create_unique( - cur_pipe->num_tasks(), _num_instances, + std::max(cur_pipe->num_tasks(), _num_instances), _num_instances, _runtime_state->query_options().__isset.local_exchange_free_blocks_limit ? _runtime_state->query_options().local_exchange_free_blocks_limit : 0); @@ -907,9 +907,13 @@ Status PipelineFragmentContext::_add_local_exchange( << " cur_pipe->operators().size(): " << cur_pipe->operators().size() << " new_pip->operators().size(): " << new_pip->operators().size(); - // Add passthrough local exchanger if necessary + // There are some local shuffles with relatively heavy operations on the sink. + // If the local sink concurrency is 1 and the local source concurrency is n, the sink becomes a bottleneck. + // Therefore, local passthrough is used to increase the concurrency of the sink. + // op -> local sink(1) -> local source (n) + // op -> local passthrough(1) -> local passthrough(n) -> local sink(n) -> local source (n) if (cur_pipe->num_tasks() > 1 && new_pip->num_tasks() == 1 && - Pipeline::is_hash_exchange(data_distribution.distribution_type)) { + Pipeline::heavy_operations_on_the_sink(data_distribution.distribution_type)) { RETURN_IF_ERROR(_add_local_exchange_impl( new_pip->operators().size(), pool, new_pip, add_pipeline(new_pip, pip_idx + 2), DataDistribution(ExchangeType::PASSTHROUGH), do_local_exchange, num_buckets, diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index 98feb85ad6b9c2..9cef1ebd2f6e12 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -30,12 +30,10 @@ #include #include -#include "arrow/record_batch.h" #include "arrow/type_fwd.h" #include "pipeline/dependency.h" #include "runtime/thread_context.h" #include "util/runtime_profile.h" -#include "util/string_util.h" #include "util/thrift_util.h" #include "vec/core/block.h" @@ -149,8 +147,8 @@ void GetArrowResultBatchCtx::on_data( delete this; } -BufferControlBlock::BufferControlBlock(const TUniqueId& id, int buffer_size, RuntimeState* state) - : _fragment_id(id), +BufferControlBlock::BufferControlBlock(TUniqueId id, int buffer_size, RuntimeState* state) + : _fragment_id(std::move(id)), _is_close(false), _is_cancelled(false), _buffer_limit(buffer_size), @@ -292,6 +290,9 @@ Status BufferControlBlock::get_arrow_batch(std::shared_ptr* r _arrow_data_arrival.wait_for(l, std::chrono::milliseconds(20)); } + if (!_status.ok()) { + return _status; + } if (_is_cancelled) { return Status::Cancelled(fmt::format("Cancelled ()", print_id(_fragment_id))); } @@ -311,9 +312,12 @@ Status BufferControlBlock::get_arrow_batch(std::shared_ptr* r // normal path end if (_is_close) { + if (!_status.ok()) { + return _status; + } std::stringstream ss; _profile.pretty_print(&ss); - VLOG_NOTICE << fmt::format( + LOG(INFO) << fmt::format( "BufferControlBlock finished, fragment_id={}, is_close={}, is_cancelled={}, " "packet_num={}, peak_memory_usage={}, profile={}", print_id(_fragment_id), _is_close, _is_cancelled, _packet_num, @@ -321,7 +325,7 @@ Status BufferControlBlock::get_arrow_batch(std::shared_ptr* r return Status::OK(); } return Status::InternalError( - fmt::format("Get Arrow Batch Abnormal Ending ()", print_id(_fragment_id))); + fmt::format("Get Arrow Batch Abnormal Ending (), ()", print_id(_fragment_id), _status)); } void BufferControlBlock::get_arrow_batch(GetArrowResultBatchCtx* ctx) { @@ -354,10 +358,14 @@ void BufferControlBlock::get_arrow_batch(GetArrowResultBatchCtx* ctx) { // normal path end if (_is_close) { + if (!_status.ok()) { + ctx->on_failure(_status); + return; + } ctx->on_close(_packet_num); std::stringstream ss; _profile.pretty_print(&ss); - VLOG_NOTICE << fmt::format( + LOG(INFO) << fmt::format( "BufferControlBlock finished, fragment_id={}, is_close={}, is_cancelled={}, " "packet_num={}, peak_memory_usage={}, profile={}", print_id(_fragment_id), _is_close, _is_cancelled, _packet_num, @@ -391,8 +399,8 @@ Status BufferControlBlock::find_arrow_schema(std::shared_ptr* arr if (_is_close) { return Status::RuntimeError(fmt::format("Closed ()", print_id(_fragment_id))); } - return Status::InternalError( - fmt::format("Get Arrow Schema Abnormal Ending ()", print_id(_fragment_id))); + return Status::InternalError(fmt::format("Get Arrow Schema Abnormal Ending (), ()", + print_id(_fragment_id), _status)); } Status BufferControlBlock::close(const TUniqueId& id, Status exec_status) { diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h index a75b670836d121..7eb0ffad0881f3 100644 --- a/be/src/runtime/buffer_control_block.h +++ b/be/src/runtime/buffer_control_block.h @@ -21,10 +21,10 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -34,7 +34,6 @@ #include "common/status.h" #include "runtime/query_statistics.h" #include "runtime/runtime_state.h" -#include "util/hash_util.hpp" namespace google::protobuf { class Closure; @@ -98,13 +97,15 @@ struct GetArrowResultBatchCtx { // buffer used for result customer and producer class BufferControlBlock { public: - BufferControlBlock(const TUniqueId& id, int buffer_size, RuntimeState* state); + BufferControlBlock(TUniqueId id, int buffer_size, RuntimeState* state); ~BufferControlBlock(); Status init(); + // try to consume _waiting_rpc or make data waiting in _fe_result_batch_queue. try to combine block to reduce rpc first. Status add_batch(RuntimeState* state, std::unique_ptr& result); Status add_arrow_batch(RuntimeState* state, std::shared_ptr& result); + // if there's Block waiting in _fe_result_batch_queue, send it(by on_data). otherwise make a rpc wait in _waiting_rpc. void get_batch(GetResultBatchCtx* ctx); // for ArrowFlightBatchLocalReader Status get_arrow_batch(std::shared_ptr* result, @@ -150,7 +151,7 @@ class BufferControlBlock { const int _buffer_limit; int64_t _packet_num; - // blocking queue for batch + // Producer. blocking queue for result batch waiting to sent to FE by _waiting_rpc. FeResultQueue _fe_result_batch_queue; ArrowFlightResultQueue _arrow_flight_result_batch_queue; // for arrow flight @@ -163,6 +164,7 @@ class BufferControlBlock { // TODO, waiting for data will block pipeline, so use a request pool to save requests waiting for data. std::condition_variable _arrow_data_arrival; + // Consumer. RPCs which FE waiting for result. when _fe_result_batch_queue filled, the rpc could be sent. std::deque _waiting_rpc; std::deque _waiting_arrow_result_batch_rpc; diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 1b92a6b3659e3f..53c1426adbd84c 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -110,6 +110,7 @@ class LookupConnectionCache; class RowCache; class DummyLRUCache; class CacheManager; +class ProcessProfile; class WalManager; class DNSCache; @@ -274,6 +275,7 @@ class ExecEnv { void set_inverted_index_searcher_cache( segment_v2::InvertedIndexSearcherCache* inverted_index_searcher_cache); void set_cache_manager(CacheManager* cm) { this->_cache_manager = cm; } + void set_process_profile(ProcessProfile* pp) { this->_process_profile = pp; } void set_tablet_schema_cache(TabletSchemaCache* c) { this->_tablet_schema_cache = c; } void set_tablet_column_object_pool(TabletColumnObjectPool* c) { this->_tablet_column_object_pool = c; @@ -311,6 +313,7 @@ class ExecEnv { LookupConnectionCache* get_lookup_connection_cache() { return _lookup_connection_cache; } RowCache* get_row_cache() { return _row_cache; } CacheManager* get_cache_manager() { return _cache_manager; } + ProcessProfile* get_process_profile() { return _process_profile; } segment_v2::InvertedIndexSearcherCache* get_inverted_index_searcher_cache() { return _inverted_index_searcher_cache; } @@ -453,6 +456,7 @@ class ExecEnv { LookupConnectionCache* _lookup_connection_cache = nullptr; RowCache* _row_cache = nullptr; CacheManager* _cache_manager = nullptr; + ProcessProfile* _process_profile = nullptr; segment_v2::InvertedIndexSearcherCache* _inverted_index_searcher_cache = nullptr; segment_v2::InvertedIndexQueryCache* _inverted_index_query_cache = nullptr; QueryCache* _query_cache = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 93e6719e1ec3ff..1fbb93f50a0cc4 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -75,6 +75,7 @@ #include "runtime/memory/mem_tracker.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/memory/thread_mem_tracker_mgr.h" +#include "runtime/process_profile.h" #include "runtime/result_buffer_mgr.h" #include "runtime/result_queue_mgr.h" #include "runtime/routine_load/routine_load_task_executor.h" @@ -449,6 +450,7 @@ Status ExecEnv::_init_mem_env() { bool is_percent = false; std::stringstream ss; // 1. init mem tracker + _process_profile = ProcessProfile::create_global_instance(); init_mem_tracker(); thread_context()->thread_mem_tracker_mgr->init(); #if defined(USE_MEM_TRACKER) && !defined(__SANITIZE_ADDRESS__) && !defined(ADDRESS_SANITIZER) && \ @@ -814,6 +816,8 @@ void ExecEnv::destroy() { // dns cache is a global instance and need to be released at last SAFE_DELETE(_dns_cache); + SAFE_DELETE(_process_profile); + _s_tracking_memory = false; LOG(INFO) << "Doris exec envorinment is destoried."; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 337a7aa41fc16b..a5392276282c87 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -33,17 +33,16 @@ #include #include #include -#include #include #include #include #include #include -#include #include #include -#include +#include +#include #include "common/status.h" // IWYU pragma: no_include @@ -57,19 +56,16 @@ #include #include -#include "cloud/config.h" #include "common/config.h" #include "common/logging.h" #include "common/object_pool.h" #include "common/utils.h" -#include "gutil/strings/substitute.h" #include "io/fs/stream_load_pipe.h" #include "pipeline/pipeline_fragment_context.h" #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/primitive_type.h" #include "runtime/query_context.h" #include "runtime/runtime_filter_mgr.h" @@ -87,24 +83,20 @@ #include "util/debug_points.h" #include "util/debug_util.h" #include "util/doris_metrics.h" -#include "util/hash_util.hpp" -#include "util/mem_info.h" #include "util/network_util.h" -#include "util/pretty_printer.h" #include "util/runtime_profile.h" #include "util/thread.h" #include "util/threadpool.h" #include "util/thrift_util.h" #include "util/uid_util.h" -#include "util/url_coding.h" #include "vec/runtime/shared_hash_table_controller.h" -#include "vec/runtime/vdatetime_value.h" namespace doris { DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(fragment_instance_count, MetricUnit::NOUNIT); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(timeout_canceled_fragment_count, MetricUnit::NOUNIT); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(fragment_thread_pool_queue_size, MetricUnit::NOUNIT); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(fragment_thread_pool_num_active_threads, MetricUnit::NOUNIT); bvar::LatencyRecorder g_fragmentmgr_prepare_latency("doris_FragmentMgr", "prepare"); bvar::Adder g_fragment_executing_count("fragment_executing_count"); @@ -182,7 +174,7 @@ static Status _do_fetch_running_queries_rpc(const FrontendInfo& fe_info, } // Avoid logic error in frontend. - if (rpc_result.__isset.status == false || rpc_result.status.status_code != TStatusCode::OK) { + if (!rpc_result.__isset.status || rpc_result.status.status_code != TStatusCode::OK) { LOG_WARNING("Failed to fetch running queries from {}, reason: {}", PrintThriftNetworkAddress(fe_info.info.coordinator_address), doris::to_string(rpc_result.status.status_code)); @@ -191,7 +183,7 @@ static Status _do_fetch_running_queries_rpc(const FrontendInfo& fe_info, doris::to_string(rpc_result.status.status_code)); } - if (rpc_result.__isset.running_queries == false) { + if (!rpc_result.__isset.running_queries) { return Status::InternalError("Failed to fetch running queries from {}, reason: {}", PrintThriftNetworkAddress(fe_info.info.coordinator_address), "running_queries is not set"); @@ -252,6 +244,8 @@ FragmentMgr::FragmentMgr(ExecEnv* exec_env) REGISTER_HOOK_METRIC(fragment_thread_pool_queue_size, [this]() { return _thread_pool->get_queue_size(); }); + REGISTER_HOOK_METRIC(fragment_thread_pool_num_active_threads, + [this]() { return _thread_pool->num_active_threads(); }); CHECK(s.ok()) << s.to_string(); } @@ -260,6 +254,7 @@ FragmentMgr::~FragmentMgr() = default; void FragmentMgr::stop() { DEREGISTER_HOOK_METRIC(fragment_instance_count); DEREGISTER_HOOK_METRIC(fragment_thread_pool_queue_size); + DEREGISTER_HOOK_METRIC(fragment_thread_pool_num_active_threads); _stop_background_threads_latch.count_down(); if (_cancel_thread) { _cancel_thread->join(); @@ -267,8 +262,11 @@ void FragmentMgr::stop() { // Only me can delete { - std::lock_guard lock(_lock); + std::unique_lock lock(_query_ctx_map_mutex); _query_ctx_map.clear(); + } + { + std::unique_lock lock(_pipeline_map_mutex); _pipeline_map.clear(); } _thread_pool->shutdown(); @@ -620,11 +618,7 @@ Status FragmentMgr::start_query_execution(const PExecPlanFragmentStartRequest* r TUniqueId query_id; query_id.__set_hi(request->query_id().hi()); query_id.__set_lo(request->query_id().lo()); - std::shared_ptr q_ctx = nullptr; - { - std::lock_guard lock(_lock); - q_ctx = _get_or_erase_query_ctx(query_id); - } + auto q_ctx = get_query_ctx(query_id); if (q_ctx) { q_ctx->set_ready_to_execute(Status::OK()); LOG_INFO("Query {} start execution", print_id(query_id)); @@ -639,116 +633,110 @@ Status FragmentMgr::start_query_execution(const PExecPlanFragmentStartRequest* r void FragmentMgr::remove_pipeline_context( std::shared_ptr f_context) { - { - std::lock_guard lock(_lock); - auto query_id = f_context->get_query_id(); - int64 now = duration_cast( - std::chrono::system_clock::now().time_since_epoch()) - .count(); - g_fragment_executing_count << -1; - g_fragment_last_active_time.set_value(now); - // this log will show when a query is really finished in BEs - LOG_INFO("Removing query {} fragment {}", print_id(query_id), f_context->get_fragment_id()); - _pipeline_map.erase({query_id, f_context->get_fragment_id()}); - } + auto query_id = f_context->get_query_id(); + int64 now = duration_cast( + std::chrono::system_clock::now().time_since_epoch()) + .count(); + g_fragment_executing_count << -1; + g_fragment_last_active_time.set_value(now); + + // this log will show when a query is really finished in BEs + LOG_INFO("Removing query {} fragment {}", print_id(query_id), f_context->get_fragment_id()); + + std::unique_lock lock(_pipeline_map_mutex); + _pipeline_map.erase({query_id, f_context->get_fragment_id()}); } -std::shared_ptr FragmentMgr::_get_or_erase_query_ctx(const TUniqueId& query_id) { +std::shared_ptr FragmentMgr::get_query_ctx(const TUniqueId& query_id) { + std::shared_lock lock(_query_ctx_map_mutex); auto search = _query_ctx_map.find(query_id); if (search != _query_ctx_map.end()) { if (auto q_ctx = search->second.lock()) { return q_ctx; - } else { - LOG(WARNING) << "Query context (query id = " << print_id(query_id) - << ") has been released."; - _query_ctx_map.erase(search); - return nullptr; } } return nullptr; } -std::shared_ptr FragmentMgr::get_or_erase_query_ctx_with_lock( - const TUniqueId& query_id) { - std::unique_lock lock(_lock); - return _get_or_erase_query_ctx(query_id); -} - -template -Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, bool pipeline, - QuerySource query_source, - std::shared_ptr& query_ctx) { +Status FragmentMgr::_get_or_create_query_ctx(const TPipelineFragmentParams& params, + TUniqueId query_id, bool pipeline, + QuerySource query_source, + std::shared_ptr& query_ctx) { DBUG_EXECUTE_IF("FragmentMgr._get_query_ctx.failed", { return Status::InternalError("FragmentMgr._get_query_ctx.failed, query id {}", print_id(query_id)); }); + + // Find _query_ctx_map, in case some other request has already + // create the query fragments context. + query_ctx = get_query_ctx(query_id); if (params.is_simplified_param) { // Get common components from _query_ctx_map - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { - query_ctx = q_ctx; - } else { + if (!query_ctx) { return Status::InternalError( "Failed to get query fragments context. Query {} may be timeout or be " "cancelled. host: {}", print_id(query_id), BackendOptions::get_localhost()); } } else { - // Find _query_ctx_map, in case some other request has already - // create the query fragments context. - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { - query_ctx = q_ctx; - return Status::OK(); - } + if (!query_ctx) { + std::unique_lock lock(_query_ctx_map_mutex); + // Only one thread need create query ctx. other thread just get query_ctx in _query_ctx_map. + auto search = _query_ctx_map.find(query_id); + if (search != _query_ctx_map.end()) { + query_ctx = search->second.lock(); + } - // First time a fragment of a query arrived. print logs. - LOG(INFO) << "query_id: " << print_id(query_id) << ", coord_addr: " << params.coord - << ", total fragment num on current host: " << params.fragment_num_on_host - << ", fe process uuid: " << params.query_options.fe_process_uuid - << ", query type: " << params.query_options.query_type - << ", report audit fe:" << params.current_connect_fe; - - // This may be a first fragment request of the query. - // Create the query fragments context. - query_ctx = QueryContext::create_shared(query_id, _exec_env, params.query_options, - params.coord, pipeline, params.is_nereids, - params.current_connect_fe, query_source); - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(query_ctx->query_mem_tracker); - RETURN_IF_ERROR(DescriptorTbl::create(&(query_ctx->obj_pool), params.desc_tbl, - &(query_ctx->desc_tbl))); - // set file scan range params - if (params.__isset.file_scan_params) { - query_ctx->file_scan_range_params_map = params.file_scan_params; - } + if (!query_ctx) { + // First time a fragment of a query arrived. print logs. + LOG(INFO) << "query_id: " << print_id(query_id) << ", coord_addr: " << params.coord + << ", total fragment num on current host: " << params.fragment_num_on_host + << ", fe process uuid: " << params.query_options.fe_process_uuid + << ", query type: " << params.query_options.query_type + << ", report audit fe:" << params.current_connect_fe; + + // This may be a first fragment request of the query. + // Create the query fragments context. + query_ctx = QueryContext::create_shared(query_id, _exec_env, params.query_options, + params.coord, pipeline, params.is_nereids, + params.current_connect_fe, query_source); + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(query_ctx->query_mem_tracker); + RETURN_IF_ERROR(DescriptorTbl::create(&(query_ctx->obj_pool), params.desc_tbl, + &(query_ctx->desc_tbl))); + // set file scan range params + if (params.__isset.file_scan_params) { + query_ctx->file_scan_range_params_map = params.file_scan_params; + } - query_ctx->query_globals = params.query_globals; + query_ctx->query_globals = params.query_globals; - if (params.__isset.resource_info) { - query_ctx->user = params.resource_info.user; - query_ctx->group = params.resource_info.group; - query_ctx->set_rsc_info = true; - } + if (params.__isset.resource_info) { + query_ctx->user = params.resource_info.user; + query_ctx->group = params.resource_info.group; + query_ctx->set_rsc_info = true; + } - _set_scan_concurrency(params, query_ctx.get()); - - if (params.__isset.workload_groups && !params.workload_groups.empty()) { - uint64_t tg_id = params.workload_groups[0].id; - WorkloadGroupPtr workload_group_ptr = - _exec_env->workload_group_mgr()->get_task_group_by_id(tg_id); - if (workload_group_ptr != nullptr) { - RETURN_IF_ERROR(workload_group_ptr->add_query(query_id, query_ctx)); - RETURN_IF_ERROR(query_ctx->set_workload_group(workload_group_ptr)); - _exec_env->runtime_query_statistics_mgr()->set_workload_group_id(print_id(query_id), - tg_id); - } else { - LOG(WARNING) << "Query/load id: " << print_id(query_ctx->query_id()) - << "can't find its workload group " << tg_id; + _set_scan_concurrency(params, query_ctx.get()); + + if (params.__isset.workload_groups && !params.workload_groups.empty()) { + uint64_t tg_id = params.workload_groups[0].id; + WorkloadGroupPtr workload_group_ptr = + _exec_env->workload_group_mgr()->get_task_group_by_id(tg_id); + if (workload_group_ptr != nullptr) { + RETURN_IF_ERROR(workload_group_ptr->add_query(query_id, query_ctx)); + RETURN_IF_ERROR(query_ctx->set_workload_group(workload_group_ptr)); + _exec_env->runtime_query_statistics_mgr()->set_workload_group_id( + print_id(query_id), tg_id); + } else { + LOG(WARNING) << "Query/load id: " << print_id(query_ctx->query_id()) + << "can't find its workload group " << tg_id; + } + } + // There is some logic in query ctx's dctor, we could not check if exists and delete the + // temp query ctx now. For example, the query id maybe removed from workload group's queryset. + _query_ctx_map.insert({query_id, query_ctx}); } } - // There is some logic in query ctx's dctor, we could not check if exists and delete the - // temp query ctx now. For example, the query id maybe removed from workload group's queryset. - _query_ctx_map.insert(std::make_pair(query_ctx->query_id(), query_ctx)); } return Status::OK(); } @@ -762,13 +750,13 @@ std::string FragmentMgr::dump_pipeline_tasks(int64_t duration) { fmt::memory_buffer debug_string_buffer; size_t i = 0; { - std::lock_guard lock(_lock); fmt::format_to(debug_string_buffer, "{} pipeline fragment contexts are still running! duration_limit={}\n", _pipeline_map.size(), duration); - timespec now; clock_gettime(CLOCK_MONOTONIC, &now); + + std::shared_lock lock(_pipeline_map_mutex); for (auto& it : _pipeline_map) { auto elapsed = it.second->elapsed_time() / 1000000000.0; if (elapsed < duration) { @@ -787,7 +775,7 @@ std::string FragmentMgr::dump_pipeline_tasks(int64_t duration) { } std::string FragmentMgr::dump_pipeline_tasks(TUniqueId& query_id) { - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { return q_ctx->print_all_pipeline_context(); } else { return fmt::format( @@ -806,7 +794,8 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, << apache::thrift::ThriftDebugString(params.query_options).c_str(); std::shared_ptr query_ctx; - RETURN_IF_ERROR(_get_query_ctx(params, params.query_id, true, query_source, query_ctx)); + RETURN_IF_ERROR( + _get_or_create_query_ctx(params, params.query_id, true, query_source, query_ctx)); SCOPED_ATTACH_TASK(query_ctx.get()); int64_t duration_ns = 0; std::shared_ptr context = @@ -839,16 +828,8 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, } { - // (query_id, fragment_id) is executed only on one BE, locks _pipeline_map. - std::lock_guard lock(_lock); for (const auto& local_param : params.local_params) { const TUniqueId& fragment_instance_id = local_param.fragment_instance_id; - auto iter = _pipeline_map.find({params.query_id, params.fragment_id}); - if (iter != _pipeline_map.end()) { - return Status::InternalError( - "exec_plan_fragment query_id({}) input duplicated fragment_id({})", - print_id(params.query_id), params.fragment_id); - } query_ctx->fragment_instance_ids.push_back(fragment_instance_id); } @@ -857,7 +838,15 @@ Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, .count(); g_fragment_executing_count << 1; g_fragment_last_active_time.set_value(now); - // TODO: simplify this mapping + + // (query_id, fragment_id) is executed only on one BE, locks _pipeline_map. + std::unique_lock lock(_pipeline_map_mutex); + auto iter = _pipeline_map.find({params.query_id, params.fragment_id}); + if (iter != _pipeline_map.end()) { + return Status::InternalError( + "exec_plan_fragment query_id({}) input duplicated fragment_id({})", + print_id(params.query_id), params.fragment_id); + } _pipeline_map.insert({{params.query_id, params.fragment_id}, context}); } @@ -887,8 +876,7 @@ void FragmentMgr::cancel_query(const TUniqueId query_id, const Status reason) { std::shared_ptr query_ctx = nullptr; std::vector all_instance_ids; { - std::lock_guard state_lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; // Copy instanceids to avoid concurrent modification. // And to reduce the scope of lock. @@ -901,7 +889,7 @@ void FragmentMgr::cancel_query(const TUniqueId query_id, const Status reason) { } query_ctx->cancel(reason); { - std::lock_guard state_lock(_lock); + std::unique_lock l(_query_ctx_map_mutex); _query_ctx_map.erase(query_id); } LOG(INFO) << "Query " << print_id(query_id) @@ -937,7 +925,7 @@ void FragmentMgr::cancel_worker() { std::vector> ctx; { - std::lock_guard lock(_lock); + std::shared_lock lock(_pipeline_map_mutex); ctx.reserve(_pipeline_map.size()); for (auto& pipeline_itr : _pipeline_map) { ctx.push_back(pipeline_itr.second); @@ -948,21 +936,24 @@ void FragmentMgr::cancel_worker() { } { - std::lock_guard lock(_lock); - for (auto it = _query_ctx_map.begin(); it != _query_ctx_map.end();) { - if (auto q_ctx = it->second.lock()) { - if (q_ctx->is_timeout(now)) { - LOG_WARNING("Query {} is timeout", print_id(it->first)); - queries_timeout.push_back(it->first); + { + // TODO: Now only the cancel worker do the GC the _query_ctx_map. each query must + // do erase the finish query unless in _query_ctx_map. Rethink the logic is ok + std::unique_lock lock(_query_ctx_map_mutex); + for (auto it = _query_ctx_map.begin(); it != _query_ctx_map.end();) { + if (auto q_ctx = it->second.lock()) { + if (q_ctx->is_timeout(now)) { + LOG_WARNING("Query {} is timeout", print_id(it->first)); + queries_timeout.push_back(it->first); + } ++it; } else { - ++it; + it = _query_ctx_map.erase(it); } - } else { - it = _query_ctx_map.erase(it); } } + std::shared_lock lock(_query_ctx_map_mutex); // We use a very conservative cancel strategy. // 0. If there are no running frontends, do not cancel any queries. // 1. If query's process uuid is zero, do not cancel @@ -1186,7 +1177,7 @@ Status FragmentMgr::apply_filterv2(const PPublishFilterRequestV2* request, const auto& fragment_ids = request->fragment_ids(); { - std::unique_lock lock(_lock); + std::shared_lock lock(_pipeline_map_mutex); for (auto fragment_id : fragment_ids) { if (is_pipeline) { auto iter = _pipeline_map.find( @@ -1242,8 +1233,7 @@ Status FragmentMgr::send_filter_size(const PSendFilterSizeRequest* request) { TUniqueId query_id; query_id.__set_hi(queryid.hi); query_id.__set_lo(queryid.lo); - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; } else { return Status::EndOfFile( @@ -1266,8 +1256,7 @@ Status FragmentMgr::sync_filter_size(const PSyncFilterSizeRequest* request) { TUniqueId query_id; query_id.__set_hi(queryid.hi); query_id.__set_lo(queryid.lo); - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; } else { return Status::EndOfFile( @@ -1287,8 +1276,7 @@ Status FragmentMgr::merge_filter(const PMergeFilterRequest* request, TUniqueId query_id; query_id.__set_hi(queryid.hi); query_id.__set_lo(queryid.lo); - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { + if (auto q_ctx = get_query_ctx(query_id)) { query_ctx = q_ctx; } else { return Status::EndOfFile( @@ -1305,7 +1293,7 @@ Status FragmentMgr::merge_filter(const PMergeFilterRequest* request, void FragmentMgr::get_runtime_query_info(std::vector* query_info_list) { { - std::lock_guard lock(_lock); + std::unique_lock lock(_query_ctx_map_mutex); for (auto iter = _query_ctx_map.begin(); iter != _query_ctx_map.end();) { if (auto q_ctx = iter->second.lock()) { WorkloadQueryInfo workload_query_info; @@ -1328,19 +1316,9 @@ Status FragmentMgr::get_realtime_exec_status(const TUniqueId& query_id, return Status::InvalidArgument("exes_status is nullptr"); } - std::shared_ptr query_context = nullptr; - - { - std::lock_guard lock(_lock); - if (auto q_ctx = _get_or_erase_query_ctx(query_id)) { - query_context = q_ctx; - } else { - return Status::NotFound("Query {} has been released", print_id(query_id)); - } - } - + std::shared_ptr query_context = get_query_ctx(query_id); if (query_context == nullptr) { - return Status::NotFound("Query {} not found", print_id(query_id)); + return Status::NotFound("Query {} not found or released", print_id(query_id)); } *exec_status = query_context->get_realtime_exec_status(); diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h index 20b2fd8cdc2063..0e7691647dd47d 100644 --- a/be/src/runtime/fragment_mgr.h +++ b/be/src/runtime/fragment_mgr.h @@ -21,9 +21,8 @@ #include #include #include -#include -#include +#include #include #include #include @@ -133,7 +132,7 @@ class FragmentMgr : public RestMonitorIface { ThreadPool* get_thread_pool() { return _thread_pool.get(); } int32_t running_query_num() { - std::unique_lock ctx_lock(_lock); + std::shared_lock lock(_query_ctx_map_mutex); return _query_ctx_map.size(); } @@ -145,35 +144,41 @@ class FragmentMgr : public RestMonitorIface { Status get_realtime_exec_status(const TUniqueId& query_id, TReportExecStatusParams* exec_status); - std::shared_ptr get_or_erase_query_ctx_with_lock(const TUniqueId& query_id); + std::shared_ptr get_query_ctx(const TUniqueId& query_id); private: std::shared_ptr _get_or_erase_query_ctx(const TUniqueId& query_id); + struct BrpcItem { + TNetworkAddress network_address; + std::vector> queries; + }; + template void _set_scan_concurrency(const Param& params, QueryContext* query_ctx); - template - Status _get_query_ctx(const Params& params, TUniqueId query_id, bool pipeline, - QuerySource query_type, std::shared_ptr& query_ctx); + Status _get_or_create_query_ctx(const TPipelineFragmentParams& params, TUniqueId query_id, + bool pipeline, QuerySource query_type, + std::shared_ptr& query_ctx); // This is input params ExecEnv* _exec_env = nullptr; + // The lock protect the `_pipeline_map` + std::shared_mutex _pipeline_map_mutex; + // (QueryID, FragmentID) -> PipelineFragmentContext + phmap::flat_hash_map, + std::shared_ptr> + _pipeline_map; + // The lock should only be used to protect the structures in fragment manager. Has to be // used in a very small scope because it may dead lock. For example, if the _lock is used // in prepare stage, the call path is prepare --> expr prepare --> may call allocator // when allocate failed, allocator may call query_is_cancelled, query is callced will also // call _lock, so that there is dead lock. - std::mutex _lock; - - // (QueryID, FragmentID) -> PipelineFragmentContext - std::unordered_map, - std::shared_ptr> - _pipeline_map; - + std::shared_mutex _query_ctx_map_mutex; // query id -> QueryContext - std::unordered_map> _query_ctx_map; + phmap::flat_hash_map> _query_ctx_map; std::unordered_map> _bf_size_map; CountDownLatch _stop_background_threads_latch; diff --git a/be/src/runtime/load_channel.cpp b/be/src/runtime/load_channel.cpp index 1ac7753b19784b..4ff83ff93dfe6f 100644 --- a/be/src/runtime/load_channel.cpp +++ b/be/src/runtime/load_channel.cpp @@ -45,8 +45,7 @@ LoadChannel::LoadChannel(const UniqueId& load_id, int64_t timeout_s, bool is_hig _backend_id(backend_id), _enable_profile(enable_profile) { std::shared_ptr query_context = - ExecEnv::GetInstance()->fragment_mgr()->get_or_erase_query_ctx_with_lock( - _load_id.to_thrift()); + ExecEnv::GetInstance()->fragment_mgr()->get_query_ctx(_load_id.to_thrift()); std::shared_ptr mem_tracker = nullptr; WorkloadGroupPtr wg_ptr = nullptr; @@ -135,7 +134,7 @@ Status LoadChannel::open(const PTabletWriterOpenRequest& params) { _is_high_priority, _self_profile); } { - std::lock_guard l(_tablets_channels_lock); + std::lock_guard l(_tablets_channels_lock); _tablets_channels.insert({index_id, channel}); } } @@ -237,7 +236,7 @@ Status LoadChannel::_handle_eos(BaseTabletsChannel* channel, if (finished) { std::lock_guard l(_lock); { - std::lock_guard l(_tablets_channels_lock); + std::lock_guard l(_tablets_channels_lock); _tablets_channels_rows.insert(std::make_pair( index_id, std::make_pair(channel->total_received_rows(), channel->num_rows_filtered()))); @@ -263,7 +262,7 @@ void LoadChannel::_report_profile(PTabletWriterAddBlockResult* response) { _self_profile->set_timestamp(_last_updated_time); { - std::lock_guard l(_tablets_channels_lock); + std::lock_guard l(_tablets_channels_lock); for (auto& it : _tablets_channels) { it.second->refresh_profile(); } diff --git a/be/src/runtime/load_channel.h b/be/src/runtime/load_channel.h index 6c150ed74d9126..36a8f363ba9bac 100644 --- a/be/src/runtime/load_channel.h +++ b/be/src/runtime/load_channel.h @@ -104,7 +104,7 @@ class LoadChannel { std::unordered_map> _tablets_channels; // index id -> (received rows, filtered rows) std::unordered_map> _tablets_channels_rows; - SpinLock _tablets_channels_lock; + std::mutex _tablets_channels_lock; // This is to save finished channels id, to handle the retry request. std::unordered_set _finished_channel_ids; // set to true if at least one tablets channel has been opened diff --git a/be/src/runtime/load_stream.cpp b/be/src/runtime/load_stream.cpp index 752e2ff95b2917..60da45fa685fbf 100644 --- a/be/src/runtime/load_stream.cpp +++ b/be/src/runtime/load_stream.cpp @@ -428,7 +428,7 @@ LoadStream::LoadStream(PUniqueId load_id, LoadStreamMgr* load_stream_mgr, bool e TUniqueId load_tid = ((UniqueId)load_id).to_thrift(); #ifndef BE_TEST std::shared_ptr query_context = - ExecEnv::GetInstance()->fragment_mgr()->get_or_erase_query_ctx_with_lock(load_tid); + ExecEnv::GetInstance()->fragment_mgr()->get_query_ctx(load_tid); if (query_context != nullptr) { _query_thread_context = {load_tid, query_context->query_mem_tracker, query_context->workload_group()}; diff --git a/be/src/runtime/memory/global_memory_arbitrator.cpp b/be/src/runtime/memory/global_memory_arbitrator.cpp index 45d7781786f2d7..0458dd72a33a35 100644 --- a/be/src/runtime/memory/global_memory_arbitrator.cpp +++ b/be/src/runtime/memory/global_memory_arbitrator.cpp @@ -19,6 +19,7 @@ #include +#include "runtime/process_profile.h" #include "runtime/thread_context.h" namespace doris { @@ -33,7 +34,7 @@ bvar::PassiveStatus g_sys_mem_avail( "meminfo_sys_mem_avail", [](void*) { return GlobalMemoryArbitrator::sys_mem_available(); }, nullptr); -std::atomic GlobalMemoryArbitrator::_s_process_reserved_memory = 0; +std::atomic GlobalMemoryArbitrator::_process_reserved_memory = 0; std::atomic GlobalMemoryArbitrator::refresh_interval_memory_growth = 0; std::mutex GlobalMemoryArbitrator::cache_adjust_capacity_lock; std::condition_variable GlobalMemoryArbitrator::cache_adjust_capacity_cv; @@ -45,9 +46,10 @@ std::atomic GlobalMemoryArbitrator::memtable_memory_refresh_notify {false} bool GlobalMemoryArbitrator::try_reserve_process_memory(int64_t bytes) { if (sys_mem_available() - bytes < MemInfo::sys_mem_available_warning_water_mark()) { + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); return false; } - int64_t old_reserved_mem = _s_process_reserved_memory.load(std::memory_order_relaxed); + int64_t old_reserved_mem = _process_reserved_memory.load(std::memory_order_relaxed); int64_t new_reserved_mem = 0; do { new_reserved_mem = old_reserved_mem + bytes; @@ -55,15 +57,16 @@ bool GlobalMemoryArbitrator::try_reserve_process_memory(int64_t bytes) { refresh_interval_memory_growth.load(std::memory_order_relaxed) + new_reserved_mem >= MemInfo::soft_mem_limit())) { + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); return false; } - } while (!_s_process_reserved_memory.compare_exchange_weak(old_reserved_mem, new_reserved_mem, - std::memory_order_relaxed)); + } while (!_process_reserved_memory.compare_exchange_weak(old_reserved_mem, new_reserved_mem, + std::memory_order_relaxed)); return true; } void GlobalMemoryArbitrator::release_process_reserved_memory(int64_t bytes) { - _s_process_reserved_memory.fetch_sub(bytes, std::memory_order_relaxed); + _process_reserved_memory.fetch_sub(bytes, std::memory_order_relaxed); } int64_t GlobalMemoryArbitrator::sub_thread_reserve_memory(int64_t bytes) { diff --git a/be/src/runtime/memory/global_memory_arbitrator.h b/be/src/runtime/memory/global_memory_arbitrator.h index 1859f45391fca3..075113088fbc5b 100644 --- a/be/src/runtime/memory/global_memory_arbitrator.h +++ b/be/src/runtime/memory/global_memory_arbitrator.h @@ -17,7 +17,7 @@ #pragma once -#include "runtime/memory/mem_tracker_limiter.h" +#include "runtime/process_profile.h" #include "util/mem_info.h" namespace doris { @@ -107,7 +107,7 @@ class GlobalMemoryArbitrator { static void release_process_reserved_memory(int64_t bytes); static inline int64_t process_reserved_memory() { - return _s_process_reserved_memory.load(std::memory_order_relaxed); + return _process_reserved_memory.load(std::memory_order_relaxed); } // `process_memory_usage` includes all reserved memory. if a thread has `reserved_memory`, @@ -122,8 +122,12 @@ class GlobalMemoryArbitrator { if (bytes <= 0) { return false; } - return process_memory_usage() + bytes >= MemInfo::soft_mem_limit() || - sys_mem_available() - bytes < MemInfo::sys_mem_available_warning_water_mark(); + auto rt = process_memory_usage() + bytes >= MemInfo::soft_mem_limit() || + sys_mem_available() - bytes < MemInfo::sys_mem_available_warning_water_mark(); + if (rt) { + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); + } + return rt; } static bool is_exceed_hard_mem_limit(int64_t bytes = 0) { @@ -139,8 +143,12 @@ class GlobalMemoryArbitrator { // tcmalloc/jemalloc allocator cache does not participate in the mem check as part of the process physical memory. // because `new/malloc` will trigger mem hook when using tcmalloc/jemalloc allocator cache, // but it may not actually alloc physical memory, which is not expected in mem hook fail. - return process_memory_usage() + bytes >= MemInfo::mem_limit() || - sys_mem_available() - bytes < MemInfo::sys_mem_available_low_water_mark(); + auto rt = process_memory_usage() + bytes >= MemInfo::mem_limit() || + sys_mem_available() - bytes < MemInfo::sys_mem_available_low_water_mark(); + if (rt) { + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); + } + return rt; } static std::string process_mem_log_str() { @@ -192,7 +200,7 @@ class GlobalMemoryArbitrator { } private: - static std::atomic _s_process_reserved_memory; + static std::atomic _process_reserved_memory; }; } // namespace doris diff --git a/be/src/runtime/memory/mem_tracker_limiter.cpp b/be/src/runtime/memory/mem_tracker_limiter.cpp index 78e66b6a579b79..05ff13f0e7c646 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.cpp +++ b/be/src/runtime/memory/mem_tracker_limiter.cpp @@ -33,27 +33,15 @@ #include "runtime/workload_group/workload_group.h" #include "service/backend_options.h" #include "util/mem_info.h" -#include "util/perf_counters.h" #include "util/runtime_profile.h" namespace doris { static bvar::Adder memory_memtrackerlimiter_cnt("memory_memtrackerlimiter_cnt"); -static bvar::Adder memory_all_trackers_sum_bytes("memory_all_trackers_sum_bytes"); -static bvar::Adder memory_global_trackers_sum_bytes("memory_global_trackers_sum_bytes"); -static bvar::Adder memory_query_trackers_sum_bytes("memory_query_trackers_sum_bytes"); -static bvar::Adder memory_load_trackers_sum_bytes("memory_load_trackers_sum_bytes"); -static bvar::Adder memory_compaction_trackers_sum_bytes( - "memory_compaction_trackers_sum_bytes"); -static bvar::Adder memory_schema_change_trackers_sum_bytes( - "memory_schema_change_trackers_sum_bytes"); -static bvar::Adder memory_other_trackers_sum_bytes("memory_other_trackers_sum_bytes"); std::atomic mem_tracker_limiter_group_counter(0); constexpr auto GC_MAX_SEEK_TRACKER = 1000; -std::atomic MemTrackerLimiter::_enable_print_log_process_usage {true}; - // Reset before each free static std::unique_ptr free_top_memory_task_profile { std::make_unique("-")}; @@ -75,6 +63,7 @@ MemTrackerLimiter::MemTrackerLimiter(Type type, const std::string& label, int64_ _type = type; _label = label; _limit = byte_limit; + _uid = UniqueId::gen_uid(); if (_type == Type::GLOBAL) { _group_num = 0; } else { @@ -216,87 +205,38 @@ std::string MemTrackerLimiter::print_address_sanitizers() { return detail; } -MemTrackerLimiter::Snapshot MemTrackerLimiter::make_snapshot() const { - Snapshot snapshot; - snapshot.type = type_string(_type); - snapshot.label = _label; - snapshot.limit = _limit; - snapshot.cur_consumption = consumption(); - snapshot.peak_consumption = peak_consumption(); - return snapshot; -} - -MemTrackerLimiter::Snapshot MemTrackerLimiter::make_reserved_trackers_snapshot() const { - Snapshot snapshot; - snapshot.type = "reserved_memory"; - snapshot.label = _label; - snapshot.limit = -1; - snapshot.cur_consumption = reserved_consumption(); - snapshot.peak_consumption = reserved_peak_consumption(); - return snapshot; -} - -void MemTrackerLimiter::make_all_reserved_trackers_snapshots(std::vector* snapshots) { - for (auto& i : ExecEnv::GetInstance()->mem_tracker_limiter_pool) { - std::lock_guard l(i.group_lock); - for (auto trackerWptr : i.trackers) { - auto tracker = trackerWptr.lock(); - if (tracker != nullptr && tracker->reserved_consumption() != 0) { - (*snapshots).emplace_back(tracker->make_reserved_trackers_snapshot()); - } - } +RuntimeProfile* MemTrackerLimiter::make_profile(RuntimeProfile* profile) const { + RuntimeProfile* profile_snapshot = profile->create_child( + fmt::format("{}@{}@id={}", _label, type_string(_type), _uid.to_string()), true, false); + RuntimeProfile::Counter* current_usage_counter = + ADD_COUNTER(profile_snapshot, "CurrentUsage", TUnit::BYTES); + RuntimeProfile::Counter* peak_usage_counter = + ADD_COUNTER(profile_snapshot, "PeakUsage", TUnit::BYTES); + COUNTER_SET(current_usage_counter, consumption()); + COUNTER_SET(peak_usage_counter, peak_consumption()); + if (has_limit()) { + RuntimeProfile::Counter* limit_counter = + ADD_COUNTER(profile_snapshot, "Limit", TUnit::BYTES); + COUNTER_SET(limit_counter, _limit); + } + if (reserved_peak_consumption() != 0) { + RuntimeProfile::Counter* reserved_counter = + ADD_COUNTER(profile_snapshot, "ReservedMemory", TUnit::BYTES); + RuntimeProfile::Counter* reserved_peak_counter = + ADD_COUNTER(profile_snapshot, "ReservedPeakMemory", TUnit::BYTES); + COUNTER_SET(reserved_counter, reserved_consumption()); + COUNTER_SET(reserved_peak_counter, reserved_peak_consumption()); } + return profile_snapshot; } -void MemTrackerLimiter::refresh_global_counter() { - std::unordered_map type_mem_sum = { - {Type::GLOBAL, 0}, {Type::QUERY, 0}, {Type::LOAD, 0}, - {Type::COMPACTION, 0}, {Type::SCHEMA_CHANGE, 0}, {Type::OTHER, 0}}; - // always ExecEnv::ready(), because Daemon::_stop_background_threads_latch - for (auto& group : ExecEnv::GetInstance()->mem_tracker_limiter_pool) { - std::lock_guard l(group.group_lock); - for (auto trackerWptr : group.trackers) { - auto tracker = trackerWptr.lock(); - if (tracker != nullptr) { - type_mem_sum[tracker->type()] += tracker->consumption(); - } - } - } - int64_t all_trackers_mem_sum = 0; - for (auto it : type_mem_sum) { - MemTrackerLimiter::TypeMemSum[it.first].set(it.second); - - all_trackers_mem_sum += it.second; - switch (it.first) { - case Type::GLOBAL: - memory_global_trackers_sum_bytes - << it.second - memory_global_trackers_sum_bytes.get_value(); - break; - case Type::QUERY: - memory_query_trackers_sum_bytes - << it.second - memory_query_trackers_sum_bytes.get_value(); - break; - case Type::LOAD: - memory_load_trackers_sum_bytes - << it.second - memory_load_trackers_sum_bytes.get_value(); - break; - case Type::COMPACTION: - memory_compaction_trackers_sum_bytes - << it.second - memory_compaction_trackers_sum_bytes.get_value(); - break; - case Type::SCHEMA_CHANGE: - memory_schema_change_trackers_sum_bytes - << it.second - memory_schema_change_trackers_sum_bytes.get_value(); - break; - case Type::OTHER: - memory_other_trackers_sum_bytes - << it.second - memory_other_trackers_sum_bytes.get_value(); - } - } - all_trackers_mem_sum += MemInfo::allocator_cache_mem(); - all_trackers_mem_sum += MemInfo::allocator_metadata_mem(); - memory_all_trackers_sum_bytes << all_trackers_mem_sum - - memory_all_trackers_sum_bytes.get_value(); +std::string MemTrackerLimiter::make_profile_str() const { + std::unique_ptr profile_snapshot = + std::make_unique("MemTrackerSnapshot"); + make_profile(profile_snapshot.get()); + std::stringstream ss; + profile_snapshot->pretty_print(&ss); + return ss.str(); } void MemTrackerLimiter::clean_tracker_limiter_group() { @@ -317,78 +257,15 @@ void MemTrackerLimiter::clean_tracker_limiter_group() { #endif } -void MemTrackerLimiter::make_process_snapshots(std::vector* snapshots) { - MemTrackerLimiter::refresh_global_counter(); - int64_t all_trackers_mem_sum = 0; - Snapshot snapshot; - for (const auto& it : MemTrackerLimiter::TypeMemSum) { - snapshot.type = "overview"; - snapshot.label = type_string(it.first); - snapshot.limit = -1; - snapshot.cur_consumption = it.second.current_value(); - snapshot.peak_consumption = it.second.peak_value(); - (*snapshots).emplace_back(snapshot); - all_trackers_mem_sum += it.second.current_value(); - } - - snapshot.type = "overview"; - snapshot.label = "tc/jemalloc_cache"; - snapshot.limit = -1; - snapshot.cur_consumption = MemInfo::allocator_cache_mem(); - snapshot.peak_consumption = -1; - (*snapshots).emplace_back(snapshot); - all_trackers_mem_sum += MemInfo::allocator_cache_mem(); - - snapshot.type = "overview"; - snapshot.label = "tc/jemalloc_metadata"; - snapshot.limit = -1; - snapshot.cur_consumption = MemInfo::allocator_metadata_mem(); - snapshot.peak_consumption = -1; - (*snapshots).emplace_back(snapshot); - all_trackers_mem_sum += MemInfo::allocator_metadata_mem(); - - snapshot.type = "overview"; - snapshot.label = "reserved_memory"; - snapshot.limit = -1; - snapshot.cur_consumption = GlobalMemoryArbitrator::process_reserved_memory(); - snapshot.peak_consumption = -1; - (*snapshots).emplace_back(snapshot); - - snapshot.type = "overview"; - snapshot.label = "sum_of_all_trackers"; // is virtual memory - snapshot.limit = -1; - snapshot.cur_consumption = all_trackers_mem_sum; - snapshot.peak_consumption = -1; - (*snapshots).emplace_back(snapshot); - - snapshot.type = "overview"; -#ifdef ADDRESS_SANITIZER - snapshot.label = "[ASAN]VmRSS(process resident memory)"; // from /proc VmRSS VmHWM -#else - snapshot.label = "VmRSS(process resident memory)"; // from /proc VmRSS VmHWM -#endif - snapshot.limit = -1; - snapshot.cur_consumption = PerfCounters::get_vm_rss(); - snapshot.peak_consumption = PerfCounters::get_vm_hwm(); - (*snapshots).emplace_back(snapshot); - - snapshot.type = "overview"; - snapshot.label = "VmSize(process virtual memory)"; // from /proc VmSize VmPeak - snapshot.limit = -1; - snapshot.cur_consumption = PerfCounters::get_vm_size(); - snapshot.peak_consumption = PerfCounters::get_vm_peak(); - (*snapshots).emplace_back(snapshot); -} - -void MemTrackerLimiter::make_type_snapshots(std::vector* snapshots, - MemTrackerLimiter::Type type) { +void MemTrackerLimiter::make_type_trackers_profile(RuntimeProfile* profile, + MemTrackerLimiter::Type type) { if (type == Type::GLOBAL) { std::lock_guard l( ExecEnv::GetInstance()->mem_tracker_limiter_pool[0].group_lock); for (auto trackerWptr : ExecEnv::GetInstance()->mem_tracker_limiter_pool[0].trackers) { auto tracker = trackerWptr.lock(); if (tracker != nullptr) { - (*snapshots).emplace_back(tracker->make_snapshot()); + tracker->make_profile(profile); } } } else { @@ -398,125 +275,80 @@ void MemTrackerLimiter::make_type_snapshots(std::vector* snapshots, for (auto trackerWptr : ExecEnv::GetInstance()->mem_tracker_limiter_pool[i].trackers) { auto tracker = trackerWptr.lock(); if (tracker != nullptr && tracker->type() == type) { - (*snapshots).emplace_back(tracker->make_snapshot()); + tracker->make_profile(profile); } } } } } -void MemTrackerLimiter::make_top_consumption_snapshots(std::vector* snapshots, - int top_num) { - std::priority_queue max_pq; - // not include global type. +std::string MemTrackerLimiter::make_type_trackers_profile_str(MemTrackerLimiter::Type type) { + std::unique_ptr profile_snapshot = + std::make_unique("TypeMemTrackersSnapshot"); + make_type_trackers_profile(profile_snapshot.get(), type); + std::stringstream ss; + profile_snapshot->pretty_print(&ss); + return ss.str(); +} + +void MemTrackerLimiter::make_top_consumption_tasks_tracker_profile(RuntimeProfile* profile, + int top_num) { + std::unique_ptr tmp_profile_snapshot = + std::make_unique("tmpSnapshot"); + std::priority_queue> max_pq; + // start from 2, not include global type. for (unsigned i = 1; i < ExecEnv::GetInstance()->mem_tracker_limiter_pool.size(); ++i) { std::lock_guard l( ExecEnv::GetInstance()->mem_tracker_limiter_pool[i].group_lock); for (auto trackerWptr : ExecEnv::GetInstance()->mem_tracker_limiter_pool[i].trackers) { auto tracker = trackerWptr.lock(); if (tracker != nullptr) { - max_pq.emplace(tracker->make_snapshot()); + auto* profile_snapshot = tracker->make_profile(tmp_profile_snapshot.get()); + max_pq.emplace(tracker->consumption(), profile_snapshot); } } } while (!max_pq.empty() && top_num > 0) { - (*snapshots).emplace_back(max_pq.top()); + RuntimeProfile* profile_snapshot = + profile->create_child(max_pq.top().second->name(), true, false); + profile_snapshot->merge(max_pq.top().second); top_num--; max_pq.pop(); } } -void MemTrackerLimiter::make_all_trackers_snapshots(std::vector* snapshots) { - for (auto& i : ExecEnv::GetInstance()->mem_tracker_limiter_pool) { - std::lock_guard l(i.group_lock); - for (auto trackerWptr : i.trackers) { - auto tracker = trackerWptr.lock(); - if (tracker != nullptr) { - (*snapshots).emplace_back(tracker->make_snapshot()); - } - } - } -} - -void MemTrackerLimiter::make_all_memory_state_snapshots(std::vector* snapshots) { - make_process_snapshots(snapshots); - make_all_trackers_snapshots(snapshots); - make_all_reserved_trackers_snapshots(snapshots); -} - -std::string MemTrackerLimiter::log_usage(Snapshot snapshot) { - return fmt::format("MemTracker Label={}, Type={}, Limit={}({} B), Used={}({} B), Peak={}({} B)", - snapshot.label, snapshot.type, MemCounter::print_bytes(snapshot.limit), - snapshot.limit, MemCounter::print_bytes(snapshot.cur_consumption), - snapshot.cur_consumption, MemCounter::print_bytes(snapshot.peak_consumption), - snapshot.peak_consumption); -} +void MemTrackerLimiter::make_all_tasks_tracker_profile(RuntimeProfile* profile) { + std::unordered_map types_profile; + types_profile[Type::QUERY] = profile->create_child("QueryTasks", true, false); + types_profile[Type::LOAD] = profile->create_child("LoadTasks", true, false); + types_profile[Type::COMPACTION] = profile->create_child("CompactionTasks", true, false); + types_profile[Type::SCHEMA_CHANGE] = profile->create_child("SchemaChangeTasks", true, false); + types_profile[Type::OTHER] = profile->create_child("OtherTasks", true, false); -std::string MemTrackerLimiter::type_log_usage(Snapshot snapshot) { - return fmt::format("Type={}, Used={}({} B), Peak={}({} B)", snapshot.type, - MemCounter::print_bytes(snapshot.cur_consumption), snapshot.cur_consumption, - MemCounter::print_bytes(snapshot.peak_consumption), - snapshot.peak_consumption); -} - -std::string MemTrackerLimiter::type_detail_usage(const std::string& msg, Type type) { - std::string detail = fmt::format("{}, Type:{}, Memory Tracker Summary", msg, type_string(type)); + // start from 2, not include global type. for (unsigned i = 1; i < ExecEnv::GetInstance()->mem_tracker_limiter_pool.size(); ++i) { std::lock_guard l( ExecEnv::GetInstance()->mem_tracker_limiter_pool[i].group_lock); for (auto trackerWptr : ExecEnv::GetInstance()->mem_tracker_limiter_pool[i].trackers) { auto tracker = trackerWptr.lock(); - if (tracker != nullptr && tracker->type() == type) { - detail += "\n " + MemTrackerLimiter::log_usage(tracker->make_snapshot()); + if (tracker != nullptr) { + tracker->make_profile(types_profile[tracker->type()]); } } } - return detail; } void MemTrackerLimiter::print_log_usage(const std::string& msg) { if (_enable_print_log_usage) { _enable_print_log_usage = false; std::string detail = msg; - detail += "\nProcess Memory Summary:\n " + GlobalMemoryArbitrator::process_mem_log_str(); - detail += "\nMemory Tracker Summary: " + log_usage(); + detail += "\nProcess Memory Summary: " + GlobalMemoryArbitrator::process_mem_log_str(); + detail += "\n" + make_profile_str(); LOG(WARNING) << detail; } } -std::string MemTrackerLimiter::log_process_usage_str() { - std::string detail; - detail += "\nProcess Memory Summary:\n " + GlobalMemoryArbitrator::process_mem_log_str(); - std::vector snapshots; - MemTrackerLimiter::make_process_snapshots(&snapshots); - MemTrackerLimiter::make_type_snapshots(&snapshots, MemTrackerLimiter::Type::GLOBAL); - MemTrackerLimiter::make_top_consumption_snapshots(&snapshots, 15); - MemTrackerLimiter::make_all_reserved_trackers_snapshots(&snapshots); - - detail += "\nMemory Tracker Summary:"; - for (const auto& snapshot : snapshots) { - if (snapshot.label.empty()) { - detail += "\n " + MemTrackerLimiter::type_log_usage(snapshot); - } else { - detail += "\n " + MemTrackerLimiter::log_usage(snapshot); - } - } - - // Add additional tracker printed when memory exceeds limit. - detail += "\n " + - ExecEnv::GetInstance()->memtable_memory_limiter()->mem_tracker()->log_usage(); - return detail; -} - -void MemTrackerLimiter::print_log_process_usage() { - // The default interval between two prints is 100ms (config::memory_maintenance_sleep_time_ms). - if (MemTrackerLimiter::_enable_print_log_process_usage) { - MemTrackerLimiter::_enable_print_log_process_usage = false; - LOG(WARNING) << log_process_usage_str(); - } -} - std::string MemTrackerLimiter::tracker_limit_exceeded_str() { std::string err_msg = fmt::format( "memory tracker limit exceeded, tracker label:{}, type:{}, limit " diff --git a/be/src/runtime/memory/mem_tracker_limiter.h b/be/src/runtime/memory/mem_tracker_limiter.h index 251a7c25a741fc..445856b1f6af83 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.h +++ b/be/src/runtime/memory/mem_tracker_limiter.h @@ -85,27 +85,47 @@ class MemTrackerLimiter final { OTHER = 5, }; - struct Snapshot { - std::string type; - std::string label; - int64_t limit = 0; - int64_t cur_consumption = 0; - int64_t peak_consumption = 0; - - bool operator<(const Snapshot& rhs) const { return cur_consumption < rhs.cur_consumption; } - }; + static std::string type_string(Type type) { + switch (type) { + case Type::GLOBAL: + return "global"; + case Type::QUERY: + return "query"; + case Type::LOAD: + return "load"; + case Type::COMPACTION: + return "compaction"; + case Type::SCHEMA_CHANGE: + return "schema_change"; + case Type::OTHER: + return "other"; + default: + LOG(FATAL) << "not match type of mem tracker limiter :" << static_cast(type); + } + LOG(FATAL) << "__builtin_unreachable"; + __builtin_unreachable(); + } - // Corresponding to MemTrackerLimiter::Type. - // MemCounter contains atomic variables, which are not allowed to be copied or moved. - inline static std::unordered_map TypeMemSum; + static std::string gc_type_string(GCType type) { + switch (type) { + case GCType::PROCESS: + return "process"; + case GCType::WORK_LOAD_GROUP: + return "work load group"; + default: + LOG(FATAL) << "not match gc type:" << static_cast(type); + } + LOG(FATAL) << "__builtin_unreachable"; + __builtin_unreachable(); + } /* * Part 2, Constructors and property methods */ - static std::shared_ptr create_shared( - MemTrackerLimiter::Type type, const std::string& label = std::string(), - int64_t byte_limit = -1); + static std::shared_ptr create_shared(MemTrackerLimiter::Type type, + const std::string& label, + int64_t byte_limit = -1); // byte_limit equal to -1 means no consumption limit, only participate in process memory statistics. MemTrackerLimiter(Type type, const std::string& label, int64_t byte_limit); @@ -119,10 +139,14 @@ class MemTrackerLimiter final { int64_t limit() const { return _limit; } bool limit_exceeded() const { return _limit >= 0 && _limit < consumption(); } Status check_limit(int64_t bytes = 0); + // Log the memory usage when memory limit is exceeded. + std::string tracker_limit_exceeded_str(); bool is_overcommit_tracker() const { return type() == Type::QUERY || type() == Type::LOAD; } bool is_query_cancelled() { return _is_query_cancelled; } void set_is_query_cancelled(bool is_cancelled) { _is_query_cancelled.store(is_cancelled); } + static void clean_tracker_limiter_group(); + /* * Part 3, Memory tracking method (use carefully!) * @@ -197,36 +221,18 @@ class MemTrackerLimiter final { DCHECK(reserved_consumption() >= 0); } - Snapshot make_reserved_trackers_snapshot() const; - static void make_all_reserved_trackers_snapshots(std::vector* snapshots); - /* - * Part 4, Memory snapshot and log method + * Part 4, Memory profile and log method */ + RuntimeProfile* make_profile(RuntimeProfile* profile) const; + std::string make_profile_str() const; + static void make_type_trackers_profile(RuntimeProfile* profile, MemTrackerLimiter::Type type); + static std::string make_type_trackers_profile_str(MemTrackerLimiter::Type type); + static void make_top_consumption_tasks_tracker_profile(RuntimeProfile* profile, int top_num); + static void make_all_tasks_tracker_profile(RuntimeProfile* profile); - static void refresh_global_counter(); - static void clean_tracker_limiter_group(); - - Snapshot make_snapshot() const; - // Returns a list of all the valid tracker snapshots. - static void make_process_snapshots(std::vector* snapshots); - static void make_type_snapshots(std::vector* snapshots, Type type); - static void make_all_trackers_snapshots(std::vector* snapshots); - static void make_all_memory_state_snapshots(std::vector* snapshots); - static void make_top_consumption_snapshots(std::vector* snapshots, int top_num); - - static std::string log_usage(Snapshot snapshot); - std::string log_usage() const { return log_usage(make_snapshot()); } - static std::string type_log_usage(Snapshot snapshot); - static std::string type_detail_usage(const std::string& msg, Type type); void print_log_usage(const std::string& msg); void enable_print_log_usage() { _enable_print_log_usage = true; } - // process memory changes more than 256M, or the GC ends - static void enable_print_log_process_usage() { _enable_print_log_process_usage = true; } - static std::string log_process_usage_str(); - static void print_log_process_usage(); - // Log the memory usage when memory limit is exceeded. - std::string tracker_limit_exceeded_str(); /* * Part 5, Memory GC method @@ -270,44 +276,6 @@ class MemTrackerLimiter final { bool is_group_commit_load {false}; private: - /* - * Part 7, Private method - */ - - static std::string type_string(Type type) { - switch (type) { - case Type::GLOBAL: - return "global"; - case Type::QUERY: - return "query"; - case Type::LOAD: - return "load"; - case Type::COMPACTION: - return "compaction"; - case Type::SCHEMA_CHANGE: - return "schema_change"; - case Type::OTHER: - return "other"; - default: - LOG(FATAL) << "not match type of mem tracker limiter :" << static_cast(type); - } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); - } - - static std::string gc_type_string(GCType type) { - switch (type) { - case GCType::PROCESS: - return "process"; - case GCType::WORK_LOAD_GROUP: - return "work load group"; - default: - LOG(FATAL) << "not match gc type:" << static_cast(type); - } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); - } - // only for Type::QUERY or Type::LOAD. static TUniqueId label_to_queryid(const std::string& label) { if (label.find("#Id=") == std::string::npos) { @@ -332,6 +300,8 @@ class MemTrackerLimiter final { // label used in the make snapshot, not guaranteed unique. std::string _label; + // For generate runtime profile, profile name must be unique. + UniqueId _uid; MemCounter _mem_counter; MemCounter _reserved_counter; @@ -351,7 +321,6 @@ class MemTrackerLimiter final { // Avoid frequent printing. bool _enable_print_log_usage = false; - static std::atomic _enable_print_log_process_usage; std::shared_ptr _query_statistics = nullptr; diff --git a/be/src/runtime/memory/memory_profile.cpp b/be/src/runtime/memory/memory_profile.cpp new file mode 100644 index 00000000000000..8dbdcbdd3af769 --- /dev/null +++ b/be/src/runtime/memory/memory_profile.cpp @@ -0,0 +1,353 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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/memory_profile.h" + +#include "bvar/reducer.h" +#include "runtime/exec_env.h" +#include "runtime/memory/global_memory_arbitrator.h" +#include "runtime/memory/mem_tracker_limiter.h" +#include "util/mem_info.h" +#include "util/runtime_profile.h" + +namespace doris { + +static bvar::Adder memory_all_tracked_sum_bytes("memory_all_tracked_sum_bytes"); +static bvar::Adder memory_global_trackers_sum_bytes("memory_global_trackers_sum_bytes"); +static bvar::Adder memory_query_trackers_sum_bytes("memory_query_trackers_sum_bytes"); +static bvar::Adder memory_load_trackers_sum_bytes("memory_load_trackers_sum_bytes"); +static bvar::Adder memory_compaction_trackers_sum_bytes( + "memory_compaction_trackers_sum_bytes"); +static bvar::Adder memory_schema_change_trackers_sum_bytes( + "memory_schema_change_trackers_sum_bytes"); +static bvar::Adder memory_other_trackers_sum_bytes("memory_other_trackers_sum_bytes"); +static bvar::Adder memory_reserved_memory_bytes("memory_reserved_memory_bytes"); +static bvar::Adder memory_all_tasks_memory_bytes("memory_all_tasks_memory_bytes"); +static bvar::Adder memory_untracked_memory_bytes("memory_untracked_memory_bytes"); + +MemoryProfile::MemoryProfile() { + _memory_overview_profile.set(std::make_unique("MemoryOverviewSnapshot")); + _global_memory_profile.set(std::make_unique("GlobalMemorySnapshot")); + _top_memory_tasks_profile.set(std::make_unique("TopMemoryTasksSnapshot")); + _tasks_memory_profile.set(std::make_unique("TasksMemorySnapshot")); +} + +void MemoryProfile::refresh_memory_overview_profile() { +#ifdef ADDRESS_SANITIZER + std::unique_ptr memory_overview_profile = + std::make_unique("[ASAN]MemoryOverviewSnapshot"); +#else + std::unique_ptr memory_overview_profile = + std::make_unique("MemoryOverviewSnapshot"); +#endif + std::unique_ptr global_memory_profile = + std::make_unique("GlobalMemorySnapshot"); + std::unique_ptr top_memory_tasks_profile = + std::make_unique("TopMemoryTasksSnapshot"); + + // 1. create profile + RuntimeProfile* untracked_memory_profile = + memory_overview_profile->create_child("UntrackedMemory", true, false); + RuntimeProfile* tracked_memory_profile = + memory_overview_profile->create_child("TrackedMemory", true, false); + RuntimeProfile* tasks_memory_overview_profile = + tracked_memory_profile->create_child("TasksMemory", true, false); + RuntimeProfile* tasks_memory_overview_details_profile = + tasks_memory_overview_profile->create_child("Details", true, false); + RuntimeProfile* global_memory_overview_profile = + tracked_memory_profile->create_child("GlobalMemory", true, false); + RuntimeProfile* jemalloc_memory_profile = + tracked_memory_profile->create_child("JemallocMemory", true, false); + RuntimeProfile* jemalloc_memory_details_profile = + jemalloc_memory_profile->create_child("Details", true, false); + + // 2. add counter + // 2.1 add process memory counter + RuntimeProfile::Counter* process_physical_memory_current_usage_counter = + ADD_COUNTER(memory_overview_profile, "PhysicalMemory(VmRSS)", TUnit::BYTES); + RuntimeProfile::Counter* process_physical_memory_peak_usage_counter = + memory_overview_profile->AddHighWaterMarkCounter("PhysicalMemoryPeak", TUnit::BYTES); + RuntimeProfile::Counter* process_virtual_memory_current_usage_counter = + ADD_COUNTER(memory_overview_profile, "VirtualMemory(VmSize)", TUnit::BYTES); + RuntimeProfile::Counter* process_virtual_memory_peak_usage_counter = + memory_overview_profile->AddHighWaterMarkCounter("VirtualMemoryPeak", TUnit::BYTES); + + // 2.2 add untracked memory counter + RuntimeProfile::Counter* untracked_memory_current_usage_counter = + ADD_COUNTER(untracked_memory_profile, "CurrentUsage", TUnit::BYTES); + RuntimeProfile::Counter* untracked_memory_peak_usage_counter = + untracked_memory_profile->AddHighWaterMarkCounter("PeakUsage", TUnit::BYTES); + + // 2.3 add tracked memory counter + RuntimeProfile::Counter* tracked_memory_current_usage_counter = + ADD_COUNTER(tracked_memory_profile, "CurrentUsage", TUnit::BYTES); + RuntimeProfile::Counter* tracked_memory_peak_usage_counter = + tracked_memory_profile->AddHighWaterMarkCounter("PeakUsage", TUnit::BYTES); + + // 2.4 add jemalloc memory counter + RuntimeProfile::Counter* jemalloc_memory_current_usage_counter = + ADD_COUNTER(jemalloc_memory_profile, "CurrentUsage", TUnit::BYTES); + RuntimeProfile::Counter* jemalloc_memory_peak_usage_counter = + jemalloc_memory_profile->AddHighWaterMarkCounter("PeakUsage", TUnit::BYTES); + RuntimeProfile::Counter* jemalloc_cache_current_usage_counter = + ADD_COUNTER(jemalloc_memory_details_profile, "Cache", TUnit::BYTES); + RuntimeProfile::Counter* jemalloc_cache_peak_usage_counter = + jemalloc_memory_details_profile->AddHighWaterMarkCounter("CachePeak", TUnit::BYTES); + RuntimeProfile::Counter* jemalloc_metadata_current_usage_counter = + ADD_COUNTER(jemalloc_memory_details_profile, "Metadata", TUnit::BYTES); + RuntimeProfile::Counter* jemalloc_metadata_peak_usage_counter = + jemalloc_memory_details_profile->AddHighWaterMarkCounter("MetadataPeak", TUnit::BYTES); + + // 2.5 add global memory counter + RuntimeProfile::Counter* global_current_usage_counter = + ADD_COUNTER(global_memory_overview_profile, "CurrentUsage", TUnit::BYTES); + RuntimeProfile::Counter* global_peak_usage_counter = + global_memory_overview_profile->AddHighWaterMarkCounter("PeakUsage", TUnit::BYTES); + + // 2.6 add tasks memory counter + RuntimeProfile::Counter* tasks_memory_current_usage_counter = + ADD_COUNTER_WITH_LEVEL(tasks_memory_overview_profile, "CurrentUsage", TUnit::BYTES, 1); + // Reserved memory is the sum of all task reserved memory, is duplicated with all task memory counter. + RuntimeProfile::Counter* reserved_memory_current_usage_counter = ADD_CHILD_COUNTER_WITH_LEVEL( + tasks_memory_overview_profile, "ReservedMemory", TUnit::BYTES, "CurrentUsage", 1); + RuntimeProfile::Counter* reserved_memory_peak_usage_counter = + tasks_memory_overview_profile->AddHighWaterMarkCounter("ReservedMemoryPeak", + TUnit::BYTES, "CurrentUsage", 1); + RuntimeProfile::Counter* tasks_memory_peak_usage_counter = + tasks_memory_overview_profile->AddHighWaterMarkCounter("PeakUsage", TUnit::BYTES); + RuntimeProfile::Counter* query_current_usage_counter = + ADD_COUNTER_WITH_LEVEL(tasks_memory_overview_details_profile, "Query", TUnit::BYTES, 1); + RuntimeProfile::Counter* query_peak_usage_counter = + tasks_memory_overview_details_profile->AddHighWaterMarkCounter( + "QueryPeak", TUnit::BYTES, "Query", 1); + RuntimeProfile::Counter* load_current_usage_counter = + ADD_COUNTER_WITH_LEVEL(tasks_memory_overview_details_profile, "Load", TUnit::BYTES, 1); + RuntimeProfile::Counter* load_peak_usage_counter = + tasks_memory_overview_details_profile->AddHighWaterMarkCounter("LoadPeak", TUnit::BYTES, + "Load", 1); + RuntimeProfile::Counter* load_all_memtables_current_usage_counter = + ADD_CHILD_COUNTER_WITH_LEVEL(tasks_memory_overview_details_profile, + "AllMemTablesMemory", TUnit::BYTES, "Load", 1); + RuntimeProfile::Counter* load_all_memtables_peak_usage_counter = + ADD_CHILD_COUNTER_WITH_LEVEL(tasks_memory_overview_details_profile, + "AllMemTablesMemoryPeak", TUnit::BYTES, "Load", 1); + RuntimeProfile::Counter* compaction_current_usage_counter = ADD_COUNTER_WITH_LEVEL( + tasks_memory_overview_details_profile, "Compaction", TUnit::BYTES, 1); + RuntimeProfile::Counter* compaction_peak_usage_counter = + tasks_memory_overview_details_profile->AddHighWaterMarkCounter( + "CompactionPeak", TUnit::BYTES, "Compaction", 1); + RuntimeProfile::Counter* schema_change_current_usage_counter = ADD_COUNTER_WITH_LEVEL( + tasks_memory_overview_details_profile, "SchemaChange", TUnit::BYTES, 1); + RuntimeProfile::Counter* schema_change_peak_usage_counter = + tasks_memory_overview_details_profile->AddHighWaterMarkCounter( + "SchemaChangePeak", TUnit::BYTES, "SchemaChange", 1); + RuntimeProfile::Counter* other_current_usage_counter = + ADD_COUNTER_WITH_LEVEL(tasks_memory_overview_details_profile, "Other", TUnit::BYTES, 1); + RuntimeProfile::Counter* other_peak_usage_counter = + tasks_memory_overview_details_profile->AddHighWaterMarkCounter( + "OtherPeak", TUnit::BYTES, "Other", 1); + // 3. refresh counter + // 3.1 refresh process memory counter + COUNTER_SET(process_physical_memory_current_usage_counter, + PerfCounters::get_vm_rss()); // from /proc VmRSS VmHWM + COUNTER_SET(process_physical_memory_peak_usage_counter, PerfCounters::get_vm_hwm()); + COUNTER_SET(process_virtual_memory_current_usage_counter, + PerfCounters::get_vm_size()); // from /proc VmSize VmPeak + COUNTER_SET(process_virtual_memory_peak_usage_counter, PerfCounters::get_vm_peak()); + + // 3.2 refresh tracked memory counter + std::unordered_map type_mem_sum = { + {MemTrackerLimiter::Type::GLOBAL, 0}, {MemTrackerLimiter::Type::QUERY, 0}, + {MemTrackerLimiter::Type::LOAD, 0}, {MemTrackerLimiter::Type::COMPACTION, 0}, + {MemTrackerLimiter::Type::SCHEMA_CHANGE, 0}, {MemTrackerLimiter::Type::OTHER, 0}}; + // always ExecEnv::ready(), because Daemon::_stop_background_threads_latch + for (auto& group : ExecEnv::GetInstance()->mem_tracker_limiter_pool) { + std::lock_guard l(group.group_lock); + for (auto trackerWptr : group.trackers) { + auto tracker = trackerWptr.lock(); + if (tracker != nullptr) { + type_mem_sum[tracker->type()] += tracker->consumption(); + } + } + } + + int64_t all_tracked_mem_sum = 0; + int64_t tasks_trackers_mem_sum = 0; + for (auto it : type_mem_sum) { + all_tracked_mem_sum += it.second; + switch (it.first) { + case MemTrackerLimiter::Type::GLOBAL: + COUNTER_SET(global_current_usage_counter, it.second); + COUNTER_SET(global_peak_usage_counter, it.second); + memory_global_trackers_sum_bytes + << it.second - memory_global_trackers_sum_bytes.get_value(); + break; + case MemTrackerLimiter::Type::QUERY: + COUNTER_SET(query_current_usage_counter, it.second); + COUNTER_SET(query_peak_usage_counter, it.second); + tasks_trackers_mem_sum += it.second; + memory_query_trackers_sum_bytes + << it.second - memory_query_trackers_sum_bytes.get_value(); + break; + case MemTrackerLimiter::Type::LOAD: + COUNTER_SET(load_current_usage_counter, it.second); + COUNTER_SET(load_peak_usage_counter, it.second); + tasks_trackers_mem_sum += it.second; + memory_load_trackers_sum_bytes + << it.second - memory_load_trackers_sum_bytes.get_value(); + break; + case MemTrackerLimiter::Type::COMPACTION: + COUNTER_SET(compaction_current_usage_counter, it.second); + COUNTER_SET(compaction_peak_usage_counter, it.second); + tasks_trackers_mem_sum += it.second; + memory_compaction_trackers_sum_bytes + << it.second - memory_compaction_trackers_sum_bytes.get_value(); + break; + case MemTrackerLimiter::Type::SCHEMA_CHANGE: + COUNTER_SET(schema_change_current_usage_counter, it.second); + COUNTER_SET(schema_change_peak_usage_counter, it.second); + tasks_trackers_mem_sum += it.second; + memory_schema_change_trackers_sum_bytes + << it.second - memory_schema_change_trackers_sum_bytes.get_value(); + break; + case MemTrackerLimiter::Type::OTHER: + COUNTER_SET(other_current_usage_counter, it.second); + COUNTER_SET(other_peak_usage_counter, it.second); + tasks_trackers_mem_sum += it.second; + memory_other_trackers_sum_bytes + << it.second - memory_other_trackers_sum_bytes.get_value(); + } + } + + MemTrackerLimiter::make_type_trackers_profile(global_memory_profile.get(), + MemTrackerLimiter::Type::GLOBAL); + + MemTrackerLimiter::make_top_consumption_tasks_tracker_profile(top_memory_tasks_profile.get(), + 15); + + COUNTER_SET(tasks_memory_current_usage_counter, tasks_trackers_mem_sum); + COUNTER_SET(tasks_memory_peak_usage_counter, tasks_trackers_mem_sum); + memory_all_tasks_memory_bytes << tasks_trackers_mem_sum - + memory_all_tasks_memory_bytes.get_value(); + + COUNTER_SET(reserved_memory_current_usage_counter, + GlobalMemoryArbitrator::process_reserved_memory()); + COUNTER_SET(reserved_memory_peak_usage_counter, + GlobalMemoryArbitrator::process_reserved_memory()); + memory_reserved_memory_bytes << GlobalMemoryArbitrator::process_reserved_memory() - + memory_reserved_memory_bytes.get_value(); + + all_tracked_mem_sum += MemInfo::allocator_cache_mem(); + COUNTER_SET(jemalloc_cache_current_usage_counter, + static_cast(MemInfo::allocator_cache_mem())); + COUNTER_SET(jemalloc_cache_peak_usage_counter, + static_cast(MemInfo::allocator_cache_mem())); + all_tracked_mem_sum += MemInfo::allocator_metadata_mem(); + COUNTER_SET(jemalloc_metadata_current_usage_counter, + static_cast(MemInfo::allocator_metadata_mem())); + COUNTER_SET(jemalloc_metadata_peak_usage_counter, + static_cast(MemInfo::allocator_metadata_mem())); + COUNTER_SET(jemalloc_memory_current_usage_counter, + jemalloc_cache_current_usage_counter->value() + + jemalloc_metadata_current_usage_counter->value()); + COUNTER_SET(jemalloc_memory_peak_usage_counter, + jemalloc_cache_current_usage_counter->value() + + jemalloc_metadata_current_usage_counter->value()); + + COUNTER_SET(tracked_memory_current_usage_counter, all_tracked_mem_sum); + COUNTER_SET(tracked_memory_peak_usage_counter, all_tracked_mem_sum); + memory_all_tracked_sum_bytes << all_tracked_mem_sum - memory_all_tracked_sum_bytes.get_value(); + + // 3.3 refresh untracked memory counter + int64_t untracked_memory = + process_physical_memory_current_usage_counter->value() - all_tracked_mem_sum; + COUNTER_SET(untracked_memory_current_usage_counter, untracked_memory); + COUNTER_SET(untracked_memory_peak_usage_counter, untracked_memory); + memory_untracked_memory_bytes << untracked_memory - memory_untracked_memory_bytes.get_value(); + + // 3.4 refresh additional tracker printed when memory exceeds limit. + COUNTER_SET(load_all_memtables_current_usage_counter, + ExecEnv::GetInstance()->memtable_memory_limiter()->mem_tracker()->consumption()); + COUNTER_SET( + load_all_memtables_peak_usage_counter, + ExecEnv::GetInstance()->memtable_memory_limiter()->mem_tracker()->peak_consumption()); + + // 4. reset profile + _memory_overview_profile.set(std::move(memory_overview_profile)); + _global_memory_profile.set(std::move(global_memory_profile)); + _top_memory_tasks_profile.set(std::move(top_memory_tasks_profile)); +} + +void MemoryProfile::refresh_tasks_memory_profile() { + std::unique_ptr tasks_memory_profile = + std::make_unique("AllTasksMemorySnapshot"); + MemTrackerLimiter::make_all_tasks_tracker_profile(tasks_memory_profile.get()); + _tasks_memory_profile.set(std::move(tasks_memory_profile)); +} + +void MemoryProfile::make_memory_profile(RuntimeProfile* profile) const { + RuntimeProfile* memory_profile_snapshot = profile->create_child("MemoryProfile", true, false); + + auto memory_overview_version_ptr = _memory_overview_profile.get(); + RuntimeProfile* memory_overview_profile = + memory_profile_snapshot->create_child(memory_overview_version_ptr->name(), true, false); + memory_overview_profile->merge(const_cast(memory_overview_version_ptr.get())); + + auto global_memory_version_ptr = _global_memory_profile.get(); + RuntimeProfile* global_memory_profile = + memory_profile_snapshot->create_child(global_memory_version_ptr->name(), true, false); + global_memory_profile->merge(const_cast(global_memory_version_ptr.get())); + + auto top_memory_tasks_version_ptr = _top_memory_tasks_profile.get(); + RuntimeProfile* top_memory_tasks_profile = memory_profile_snapshot->create_child( + top_memory_tasks_version_ptr->name(), true, false); + top_memory_tasks_profile->merge( + const_cast(top_memory_tasks_version_ptr.get())); + + auto tasks_memory_version_ptr = _tasks_memory_profile.get(); + RuntimeProfile* tasks_memory_profile = + memory_profile_snapshot->create_child(tasks_memory_version_ptr->name(), true, false); + tasks_memory_profile->merge(const_cast(tasks_memory_version_ptr.get())); +} + +int64_t MemoryProfile::query_current_usage() { + return memory_query_trackers_sum_bytes.get_value(); +} +int64_t MemoryProfile::load_current_usage() { + return memory_load_trackers_sum_bytes.get_value(); +} +int64_t MemoryProfile::compaction_current_usage() { + return memory_compaction_trackers_sum_bytes.get_value(); +} +int64_t MemoryProfile::schema_change_current_usage() { + return memory_schema_change_trackers_sum_bytes.get_value(); +} +int64_t MemoryProfile::other_current_usage() { + return memory_other_trackers_sum_bytes.get_value(); +} + +void MemoryProfile::print_log_process_usage() { + if (_enable_print_log_process_usage) { + _enable_print_log_process_usage = false; + LOG(WARNING) << "Process Memory Summary: " + GlobalMemoryArbitrator::process_mem_log_str(); + LOG(WARNING) << "\n" << print_memory_overview_profile(); + LOG(WARNING) << "\n" << print_global_memory_profile(); + LOG(WARNING) << "\n" << print_top_memory_tasks_profile(); + } +} + +} // namespace doris diff --git a/be/src/runtime/memory/memory_profile.h b/be/src/runtime/memory/memory_profile.h new file mode 100644 index 00000000000000..9f1bab0c02a802 --- /dev/null +++ b/be/src/runtime/memory/memory_profile.h @@ -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. + +#pragma once + +#include + +#include "util/runtime_profile.h" + +namespace doris { + +class MemoryProfile { +public: + MemoryProfile(); + + void refresh_memory_overview_profile(); + void refresh_tasks_memory_profile(); + + void make_memory_profile(RuntimeProfile* profile) const; + + std::string print_memory_overview_profile() const { + std::stringstream ss; + auto version_ptr = _memory_overview_profile.get(); + version_ptr->pretty_print(&ss); + return ss.str(); + } + + std::string print_global_memory_profile() const { + std::stringstream ss; + auto version_ptr = _global_memory_profile.get(); + version_ptr->pretty_print(&ss); + return ss.str(); + } + + std::string print_top_memory_tasks_profile() const { + std::stringstream ss; + auto version_ptr = _top_memory_tasks_profile.get(); + version_ptr->pretty_print(&ss); + return ss.str(); + } + + std::string print_tasks_memory_profile() const { + std::stringstream ss; + auto version_ptr = _tasks_memory_profile.get(); + version_ptr->pretty_print(&ss); + return ss.str(); + } + + static int64_t query_current_usage(); + static int64_t load_current_usage(); + static int64_t compaction_current_usage(); + static int64_t schema_change_current_usage(); + static int64_t other_current_usage(); + + // process memory changes more than 256M, or the GC ends + void enable_print_log_process_usage() { _enable_print_log_process_usage = true; } + void print_log_process_usage(); + +private: + MultiVersion _memory_overview_profile; + MultiVersion _global_memory_profile; + MultiVersion _top_memory_tasks_profile; + MultiVersion _tasks_memory_profile; + + std::atomic _enable_print_log_process_usage {true}; +}; + +} // namespace doris diff --git a/be/src/runtime/memory/memory_reclamation.cpp b/be/src/runtime/memory/memory_reclamation.cpp index 17f5a41f462b50..2d6098f7438759 100644 --- a/be/src/runtime/memory/memory_reclamation.cpp +++ b/be/src/runtime/memory/memory_reclamation.cpp @@ -17,7 +17,8 @@ #include "runtime/memory/memory_reclamation.h" -#include "runtime/memory/cache_manager.h" +#include "runtime/exec_env.h" +#include "runtime/memory/mem_tracker_limiter.h" #include "runtime/workload_group/workload_group.h" #include "runtime/workload_group/workload_group_manager.h" #include "util/mem_info.h" @@ -55,9 +56,15 @@ bool MemoryReclamation::process_minor_gc(std::string mem_info) { } if (config::enable_query_memory_overcommit) { - VLOG_NOTICE << MemTrackerLimiter::type_detail_usage( - "[MemoryGC] before free top memory overcommit query in minor GC", - MemTrackerLimiter::Type::QUERY); + if (config::crash_in_memory_tracker_inaccurate) { + LOG(INFO) << fmt::format( + "[MemoryGC] before free top memory overcommit query in minor GC, Type:{}, " + "Memory " + "Tracker Summary: {}", + MemTrackerLimiter::type_string(MemTrackerLimiter::Type::QUERY), + MemTrackerLimiter::make_type_trackers_profile_str( + MemTrackerLimiter::Type::QUERY)); + } RuntimeProfile* toq_profile = profile->create_child("FreeTopOvercommitMemoryQuery", true, true); freed_mem += MemTrackerLimiter::free_top_overcommit_query( @@ -98,8 +105,14 @@ bool MemoryReclamation::process_full_gc(std::string mem_info) { } } - VLOG_NOTICE << MemTrackerLimiter::type_detail_usage( - "[MemoryGC] before free top memory query in full GC", MemTrackerLimiter::Type::QUERY); + if (config::crash_in_memory_tracker_inaccurate) { + LOG(INFO) << fmt::format( + "[MemoryGC] before free top memory query in full GC, Type:{}, Memory Tracker " + "Summary: " + "{}", + MemTrackerLimiter::type_string(MemTrackerLimiter::Type::QUERY), + MemTrackerLimiter::make_type_trackers_profile_str(MemTrackerLimiter::Type::QUERY)); + } RuntimeProfile* tmq_profile = profile->create_child("FreeTopMemoryQuery", true, true); freed_mem += MemTrackerLimiter::free_top_memory_query( MemInfo::process_full_gc_size() - freed_mem, mem_info, tmq_profile); @@ -108,9 +121,14 @@ bool MemoryReclamation::process_full_gc(std::string mem_info) { } if (config::enable_query_memory_overcommit) { - VLOG_NOTICE << MemTrackerLimiter::type_detail_usage( - "[MemoryGC] before free top memory overcommit load in full GC", - MemTrackerLimiter::Type::LOAD); + if (config::crash_in_memory_tracker_inaccurate) { + LOG(INFO) << fmt::format( + "[MemoryGC] before free top memory overcommit load in full GC, Type:{}, Memory " + "Tracker Summary: {}", + MemTrackerLimiter::type_string(MemTrackerLimiter::Type::LOAD), + MemTrackerLimiter::make_type_trackers_profile_str( + MemTrackerLimiter::Type::LOAD)); + } RuntimeProfile* tol_profile = profile->create_child("FreeTopMemoryOvercommitLoad", true, true); freed_mem += MemTrackerLimiter::free_top_overcommit_load( @@ -120,8 +138,14 @@ bool MemoryReclamation::process_full_gc(std::string mem_info) { } } - VLOG_NOTICE << MemTrackerLimiter::type_detail_usage( - "[MemoryGC] before free top memory load in full GC", MemTrackerLimiter::Type::LOAD); + if (config::crash_in_memory_tracker_inaccurate) { + LOG(INFO) << fmt::format( + "[MemoryGC] before free top memory load in full GC, Type:{}, Memory Tracker " + "Summary: " + "{}", + MemTrackerLimiter::type_string(MemTrackerLimiter::Type::LOAD), + MemTrackerLimiter::make_type_trackers_profile_str(MemTrackerLimiter::Type::LOAD)); + } RuntimeProfile* tml_profile = profile->create_child("FreeTopMemoryLoad", true, true); freed_mem += MemTrackerLimiter::free_top_memory_load( MemInfo::process_full_gc_size() - freed_mem, mem_info, tml_profile); diff --git a/be/src/runtime/memory/thread_mem_tracker_mgr.h b/be/src/runtime/memory/thread_mem_tracker_mgr.h index fd14750d8b8ebc..db3b32a6298820 100644 --- a/be/src/runtime/memory/thread_mem_tracker_mgr.h +++ b/be/src/runtime/memory/thread_mem_tracker_mgr.h @@ -111,7 +111,7 @@ class ThreadMemTrackerMgr { return fmt::format( "ThreadMemTrackerMgr debug, _untracked_mem:{}, " "_limiter_tracker:<{}>, _consumer_tracker_stack:<{}>", - std::to_string(_untracked_mem), _limiter_tracker->log_usage(), + std::to_string(_untracked_mem), _limiter_tracker->make_profile_str(), fmt::to_string(consumer_tracker_buf)); } diff --git a/be/src/runtime/process_profile.cpp b/be/src/runtime/process_profile.cpp new file mode 100644 index 00000000000000..d91aedbeac2025 --- /dev/null +++ b/be/src/runtime/process_profile.cpp @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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/process_profile.h" + +#include + +#include "runtime/memory/memory_profile.h" + +namespace doris { + +ProcessProfile::ProcessProfile() { + _memory_profile = std::make_unique(); +} + +void ProcessProfile::refresh_profile() { + // 1. refresh profile + _memory_profile->refresh_memory_overview_profile(); + _memory_profile->refresh_tasks_memory_profile(); + // TODO refresh other profile + + // 2. make profile + std::unique_ptr process_profile = + std::make_unique("ProcessProfile"); + _memory_profile->make_memory_profile(process_profile.get()); + _process_profile.set(std::move(process_profile)); + // TODO make other profile +} + +} // namespace doris diff --git a/be/src/runtime/process_profile.h b/be/src/runtime/process_profile.h new file mode 100644 index 00000000000000..24b128ab5528e2 --- /dev/null +++ b/be/src/runtime/process_profile.h @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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 "runtime/exec_env.h" +#include "runtime/memory/memory_profile.h" +#include "util/runtime_profile.h" + +namespace doris { + +class ProcessProfile { +public: + static ProcessProfile* create_global_instance() { return new ProcessProfile(); } + static ProcessProfile* instance() { return ExecEnv::GetInstance()->get_process_profile(); } + ProcessProfile(); + + void refresh_profile(); + + std::string print_process_profile() const { + auto version_ptr = _process_profile.get(); + std::stringstream ss; + version_ptr->pretty_print(&ss); + return ss.str(); + } + + std::string print_process_profile_no_root() const { + std::stringstream ss; + std::vector profiles; + auto version_ptr = _process_profile.get(); + auto* process_profile = const_cast(version_ptr.get()); + process_profile->get_children(&profiles); + for (auto* profile : profiles) { + profile->pretty_print(&ss); + } + return ss.str(); + } + + MemoryProfile* memory_profile() { return _memory_profile.get(); } + +private: + MultiVersion _process_profile; + std::unique_ptr _memory_profile; +}; + +} // namespace doris diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index 06150ae3d2039d..d24146a449920b 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -42,6 +42,7 @@ #include "io/fs/multi_table_pipe.h" #include "io/fs/stream_load_pipe.h" #include "runtime/exec_env.h" +#include "runtime/memory/memory_profile.h" #include "runtime/message_body_sink.h" #include "runtime/routine_load/data_consumer.h" #include "runtime/routine_load/data_consumer_group.h" @@ -316,8 +317,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { bool RoutineLoadTaskExecutor::_reach_memory_limit() { bool is_exceed_soft_mem_limit = GlobalMemoryArbitrator::is_exceed_soft_mem_limit(); - auto current_load_mem_value = - MemTrackerLimiter::TypeMemSum[MemTrackerLimiter::Type::LOAD].current_value(); + auto current_load_mem_value = MemoryProfile::load_current_usage(); if (is_exceed_soft_mem_limit || current_load_mem_value > _load_mem_limit) { LOG(INFO) << "is_exceed_soft_mem_limit: " << is_exceed_soft_mem_limit << " current_load_mem_value: " << current_load_mem_value diff --git a/be/src/runtime/routine_load/routine_load_task_executor.h b/be/src/runtime/routine_load/routine_load_task_executor.h index 0e597d796c9f77..b1196f7824afac 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.h +++ b/be/src/runtime/routine_load/routine_load_task_executor.h @@ -73,6 +73,8 @@ class RoutineLoadTaskExecutor { std::vector* partition_offsets, int timeout); + ThreadPool& get_thread_pool() { return *_thread_pool; } + private: // execute the task void exec_task(std::shared_ptr ctx, DataConsumerPool* pool, diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 4f24824ac70547..34d9be9bcb2b4d 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -299,7 +299,7 @@ Status RuntimeState::init(const TUniqueId& fragment_instance_id, const TQueryOpt } std::weak_ptr RuntimeState::get_query_ctx_weak() { - return _exec_env->fragment_mgr()->get_or_erase_query_ctx_with_lock(_query_ctx->query_id()); + return _exec_env->fragment_mgr()->get_query_ctx(_query_ctx->query_id()); } void RuntimeState::init_mem_trackers(const std::string& name, const TUniqueId& id) { diff --git a/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp b/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp index e935aff996d55e..c24fcb73384494 100644 --- a/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp +++ b/be/src/service/arrow_flight/arrow_flight_batch_reader.cpp @@ -56,7 +56,7 @@ arrow::Status ArrowFlightBatchReaderBase::_return_invalid_status(const std::stri } ArrowFlightBatchReaderBase::~ArrowFlightBatchReaderBase() { - VLOG_NOTICE << fmt::format( + LOG(INFO) << fmt::format( "ArrowFlightBatchReader finished, packet_seq={}, result_addr={}:{}, finistId={}, " "convert_arrow_batch_timer={}, deserialize_block_timer={}, peak_memory_usage={}", _packet_seq, _statement->result_addr.hostname, _statement->result_addr.port, diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp index a74f00291de640..151f95179c90aa 100644 --- a/be/src/service/http_service.cpp +++ b/be/src/service/http_service.cpp @@ -32,6 +32,7 @@ #include "common/status.h" #include "http/action/adjust_log_level.h" #include "http/action/adjust_tracing_dump.h" +#include "http/action/batch_download_action.h" #include "http/action/be_proc_thread_action.h" #include "http/action/calc_file_crc_action.h" #include "http/action/check_rpc_channel_action.h" @@ -79,6 +80,7 @@ #include "util/doris_metrics.h" namespace doris { +#include "common/compile_check_begin.h" namespace { std::shared_ptr get_rate_limit_group(event_base* event_base) { auto rate_limit = config::download_binlog_rate_limit_kbs; @@ -290,6 +292,16 @@ void HttpService::register_local_handler(StorageEngine& engine) { tablet_download_action); _ev_http_server->register_handler(HttpMethod::GET, "/api/_tablet/_download", tablet_download_action); + + BatchDownloadAction* batch_download_action = + _pool.add(new BatchDownloadAction(_env, _rate_limit_group, allow_paths)); + _ev_http_server->register_handler(HttpMethod::HEAD, "/api/_tablet/_batch_download", + batch_download_action); + _ev_http_server->register_handler(HttpMethod::GET, "/api/_tablet/_batch_download", + batch_download_action); + _ev_http_server->register_handler(HttpMethod::POST, "/api/_tablet/_batch_download", + batch_download_action); + if (config::enable_single_replica_load) { DownloadAction* single_replica_download_action = _pool.add(new DownloadAction( _env, nullptr, allow_paths, config::single_replica_load_download_num_workers)); @@ -455,4 +467,5 @@ int HttpService::get_real_port() const { return _ev_http_server->get_real_port(); } +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index be99278ab541a3..c476398a1a4962 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -665,15 +665,11 @@ void PInternalService::cancel_plan_fragment(google::protobuf::RpcController* /*c void PInternalService::fetch_data(google::protobuf::RpcController* controller, const PFetchDataRequest* request, PFetchDataResult* result, google::protobuf::Closure* done) { - bool ret = _heavy_work_pool.try_offer([this, controller, request, result, done]() { - brpc::Controller* cntl = static_cast(controller); - GetResultBatchCtx* ctx = new GetResultBatchCtx(cntl, result, done); - _exec_env->result_mgr()->fetch_data(request->finst_id(), ctx); - }); - if (!ret) { - offer_failed(result, done, _heavy_work_pool); - return; - } + // fetch_data is a light operation which will put a request rather than wait inplace when there's no data ready. + // when there's data ready, use brpc to send. there's queue in brpc service. won't take it too long. + auto* cntl = static_cast(controller); + auto* ctx = new GetResultBatchCtx(cntl, result, done); + _exec_env->result_mgr()->fetch_data(request->finst_id(), ctx); } void PInternalService::fetch_arrow_data(google::protobuf::RpcController* controller, @@ -1238,7 +1234,10 @@ void PInternalService::report_stream_load_status(google::protobuf::RpcController void PInternalService::get_info(google::protobuf::RpcController* controller, const PProxyRequest* request, PProxyResult* response, google::protobuf::Closure* done) { - bool ret = _heavy_work_pool.try_offer([this, request, response, done]() { + bool ret = _exec_env->routine_load_task_executor()->get_thread_pool().submit_func([this, + request, + response, + done]() { brpc::ClosureGuard closure_guard(done); // PProxyRequest is defined in gensrc/proto/internal_service.proto // Currently it supports 2 kinds of requests: diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp index dd11d5ae46f740..a0cd77aee41931 100644 --- a/be/src/util/arrow/row_batch.cpp +++ b/be/src/util/arrow/row_batch.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -84,12 +85,10 @@ Status convert_to_arrow_type(const TypeDescriptor& type, std::shared_ptr MemInfo::_s_je_dirty_pages_mem = std::numeric_limits MemInfo::_s_je_dirty_pages_mem_limit = std::numeric_limits::max(); std::atomic MemInfo::_s_virtual_memory_used = 0; -int64_t MemInfo::_s_cgroup_mem_limit = std::numeric_limits::max(); -int64_t MemInfo::_s_cgroup_mem_usage = std::numeric_limits::min(); -bool MemInfo::_s_cgroup_mem_refresh_state = false; +std::atomic MemInfo::_s_cgroup_mem_limit = std::numeric_limits::max(); +std::atomic MemInfo::_s_cgroup_mem_usage = std::numeric_limits::min(); +std::atomic MemInfo::_s_cgroup_mem_refresh_state = false; int64_t MemInfo::_s_cgroup_mem_refresh_wait_times = 0; static std::unordered_map _mem_info_bytes; @@ -94,7 +94,7 @@ void MemInfo::refresh_allocator_mem() { #elif defined(USE_JEMALLOC) // jemalloc mallctl refer to : https://jemalloc.net/jemalloc.3.html // https://www.bookstack.cn/read/aliyun-rds-core/4a0cdf677f62feb3.md - // Check the Doris BE web page `http://ip:webserver_port/memz` to get the Jemalloc Profile. + // Check the Doris BE web page `http://ip:webserver_port/memory` to get the Jemalloc Profile. // 'epoch' is a special mallctl -- it updates the statistics. Without it, all // the following calls will return stale values. It increments and returns @@ -191,15 +191,18 @@ void MemInfo::refresh_proc_meminfo() { // refresh cgroup memory if (config::enable_use_cgroup_memory_info) { if (_s_cgroup_mem_refresh_wait_times >= 0) { - auto status = CGroupMemoryCtl::find_cgroup_mem_limit(&_s_cgroup_mem_limit); + int64_t cgroup_mem_limit; + auto status = CGroupMemoryCtl::find_cgroup_mem_limit(&cgroup_mem_limit); if (!status.ok()) { _s_cgroup_mem_limit = std::numeric_limits::max(); // find cgroup limit failed, wait 300s, 1000 * 100ms. _s_cgroup_mem_refresh_wait_times = -3000; - LOG(INFO) << "Refresh cgroup memory limit failed, refresh again after 300s, cgroup " - "mem limit: " - << _s_cgroup_mem_limit; + LOG(WARNING) + << "Refresh cgroup memory limit failed, refresh again after 300s, cgroup " + "mem limit: " + << _s_cgroup_mem_limit << ", " << status; } else { + _s_cgroup_mem_limit = cgroup_mem_limit; // wait 10s, 100 * 100ms, avoid too frequently. _s_cgroup_mem_refresh_wait_times = -100; } @@ -207,12 +210,19 @@ void MemInfo::refresh_proc_meminfo() { _s_cgroup_mem_refresh_wait_times++; } + // cgroup mem limit is refreshed every 10 seconds, + // cgroup mem usage is refreshed together with memInfo every time, which is very frequent. if (_s_cgroup_mem_limit != std::numeric_limits::max()) { - auto status = CGroupMemoryCtl::find_cgroup_mem_usage(&_s_cgroup_mem_usage); + int64_t cgroup_mem_usage; + auto status = CGroupMemoryCtl::find_cgroup_mem_usage(&cgroup_mem_usage); if (!status.ok()) { _s_cgroup_mem_usage = std::numeric_limits::min(); _s_cgroup_mem_refresh_state = false; + LOG_EVERY_N(WARNING, 500) + << "Refresh cgroup memory usage failed, cgroup mem limit: " + << _s_cgroup_mem_limit << ", " << status; } else { + _s_cgroup_mem_usage = cgroup_mem_usage; _s_cgroup_mem_refresh_state = true; } } else { @@ -231,7 +241,8 @@ void MemInfo::refresh_proc_meminfo() { if (physical_mem < 0) { physical_mem = _s_cgroup_mem_limit; } else { - physical_mem = std::min(physical_mem, _s_cgroup_mem_limit); + physical_mem = + std::min(physical_mem, _s_cgroup_mem_limit.load(std::memory_order_relaxed)); } } @@ -274,6 +285,12 @@ void MemInfo::refresh_proc_meminfo() { mem_available = _mem_info_bytes["MemAvailable"]; } if (_s_cgroup_mem_refresh_state) { + // Note, CgroupV2 MemAvailable is usually a little smaller than Process MemAvailable. + // Process `MemAvailable = MemFree - LowWaterMark + (PageCache - min(PageCache / 2, LowWaterMark))`, + // from `MemAvailable` in `/proc/meminfo`, calculated by OS. + // CgroupV2 `MemAvailable = cgroup_mem_limit - cgroup_mem_usage`, + // `cgroup_mem_usage = memory.current - inactive_file - slab_reclaimable`, in fact, + // there seems to be some memory that can be reused in `cgroup_mem_usage`. if (mem_available < 0) { mem_available = _s_cgroup_mem_limit - _s_cgroup_mem_usage; } else { diff --git a/be/src/util/mem_info.h b/be/src/util/mem_info.h index 10d2d086801540..a8b52cad2448d1 100644 --- a/be/src/util/mem_info.h +++ b/be/src/util/mem_info.h @@ -200,6 +200,18 @@ class MemInfo { return PrettyPrinter::print(_s_soft_mem_limit.load(std::memory_order_relaxed), TUnit::BYTES); } + static inline int64_t cgroup_mem_limit() { + DCHECK(_s_initialized); + return _s_cgroup_mem_limit.load(std::memory_order_relaxed); + } + static inline int64_t cgroup_mem_usage() { + DCHECK(_s_initialized); + return _s_cgroup_mem_usage.load(std::memory_order_relaxed); + } + static inline int64_t cgroup_mem_refresh_state() { + DCHECK(_s_initialized); + return _s_cgroup_mem_refresh_state.load(std::memory_order_relaxed); + } static std::string debug_string(); @@ -217,9 +229,9 @@ class MemInfo { static std::atomic _s_je_dirty_pages_mem_limit; static std::atomic _s_virtual_memory_used; - static int64_t _s_cgroup_mem_limit; - static int64_t _s_cgroup_mem_usage; - static bool _s_cgroup_mem_refresh_state; + static std::atomic _s_cgroup_mem_limit; + static std::atomic _s_cgroup_mem_usage; + static std::atomic _s_cgroup_mem_refresh_state; static int64_t _s_cgroup_mem_refresh_wait_times; static std::atomic _s_sys_mem_available; diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp index a9e197fba9baf6..d58dbf12452a39 100644 --- a/be/src/util/runtime_profile.cpp +++ b/be/src/util/runtime_profile.cpp @@ -274,7 +274,7 @@ void RuntimeProfile::compute_time_in_profile(int64_t total) { RuntimeProfile* RuntimeProfile::create_child(const std::string& name, bool indent, bool prepend) { std::lock_guard l(_children_lock); - DCHECK(_child_map.find(name) == _child_map.end()); + DCHECK(_child_map.find(name) == _child_map.end()) << ", name: " << name; RuntimeProfile* child = _pool->add(new RuntimeProfile(name)); if (this->is_set_metadata()) { child->set_metadata(this->metadata()); @@ -285,8 +285,8 @@ RuntimeProfile* RuntimeProfile::create_child(const std::string& name, bool inden if (_children.empty()) { add_child_unlock(child, indent, nullptr); } else { - ChildVector::iterator pos = prepend ? _children.begin() : _children.end(); - add_child_unlock(child, indent, (*pos).first); + auto* pos = prepend ? _children.begin()->first : nullptr; + add_child_unlock(child, indent, pos); } return child; } @@ -577,8 +577,6 @@ void RuntimeProfile::to_thrift(TRuntimeProfileTree* tree) { } void RuntimeProfile::to_thrift(std::vector* nodes) { - nodes->reserve(nodes->size() + _children.size()); - int index = nodes->size(); nodes->push_back(TRuntimeProfileNode()); TRuntimeProfileNode& node = (*nodes)[index]; @@ -605,10 +603,13 @@ void RuntimeProfile::to_thrift(std::vector* nodes) { ChildVector children; { + // _children may be modified during to_thrift(), + // so we have to lock and copy _children to avoid race condition std::lock_guard l(_children_lock); children = _children; } node.num_children = children.size(); + nodes->reserve(nodes->size() + children.size()); for (int i = 0; i < children.size(); ++i) { int child_idx = nodes->size(); diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index b77157d1f5b3de..955d77b72aa51c 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -51,8 +51,8 @@ class TRuntimeProfileTree; #define MACRO_CONCAT(x, y) CONCAT_IMPL(x, y) #define ADD_LABEL_COUNTER(profile, name) (profile)->add_counter(name, TUnit::NONE) -#define ADD_LABEL_COUNTER_WITH_LEVEL(profile, name, type) \ - (profile)->add_counter_with_level(name, TUnit::NONE, type) +#define ADD_LABEL_COUNTER_WITH_LEVEL(profile, name, level) \ + (profile)->add_counter_with_level(name, TUnit::NONE, level) #define ADD_COUNTER(profile, name, type) (profile)->add_counter(name, type) #define ADD_COUNTER_WITH_LEVEL(profile, name, type, level) \ (profile)->add_counter_with_level(name, type, level) diff --git a/be/src/util/threadpool.h b/be/src/util/threadpool.h index 5ce27e2f27b9a5..1c2db3d847ab61 100644 --- a/be/src/util/threadpool.h +++ b/be/src/util/threadpool.h @@ -20,12 +20,11 @@ #pragma once -#include -#include - #include #include #include +#include +#include // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -50,7 +49,7 @@ class ThreadPoolToken; class Runnable { public: virtual void run() = 0; - virtual ~Runnable() {} + virtual ~Runnable() = default; }; // ThreadPool takes a lot of arguments. We provide sane defaults with a builder. @@ -127,6 +126,9 @@ class ThreadPoolBuilder { return Status::OK(); } + ThreadPoolBuilder(const ThreadPoolBuilder&) = delete; + void operator=(const ThreadPoolBuilder&) = delete; + private: friend class ThreadPool; const std::string _name; @@ -136,9 +138,6 @@ class ThreadPoolBuilder { CgroupCpuCtl* _cgroup_cpu_ctl = nullptr; std::chrono::milliseconds _idle_timeout; - ThreadPoolBuilder(const ThreadPoolBuilder&) = delete; - void operator=(const ThreadPoolBuilder&) = delete; - template static constexpr bool always_false_v = false; }; @@ -256,13 +255,22 @@ class ThreadPool { return _total_queued_tasks; } - std::vector debug_info() { + std::vector debug_info() const { std::lock_guard l(_lock); std::vector arr = {_num_threads, static_cast(_threads.size()), _min_threads, _max_threads}; return arr; } + std::string get_info() const { + std::lock_guard l(_lock); + return fmt::format("ThreadPool(name={}, threads(active/pending)=({}/{}), queued_task={})", + _name, _active_threads, _num_threads_pending_start, _total_queued_tasks); + } + + ThreadPool(const ThreadPool&) = delete; + void operator=(const ThreadPool&) = delete; + private: friend class ThreadPoolBuilder; friend class ThreadPoolToken; @@ -372,7 +380,7 @@ class ThreadPool { // // Protected by _lock. struct IdleThread : public boost::intrusive::list_base_hook<> { - explicit IdleThread() {} + explicit IdleThread() = default; // Condition variable for "queue is not empty". Waiters wake up when a new // task is queued. @@ -384,9 +392,6 @@ class ThreadPool { // ExecutionMode::CONCURRENT token used by the pool for tokenless submission. std::unique_ptr _tokenless; - - ThreadPool(const ThreadPool&) = delete; - void operator=(const ThreadPool&) = delete; }; // Entry point for token-based task submission and blocking for a particular @@ -434,6 +439,9 @@ class ThreadPoolToken { return _entries.size(); } + ThreadPoolToken(const ThreadPoolToken&) = delete; + void operator=(const ThreadPoolToken&) = delete; + private: // All possible token states. Legal state transitions: // IDLE -> RUNNING: task is submitted via token @@ -516,9 +524,6 @@ class ThreadPoolToken { int _num_submitted_tasks; // Number of tasks which has not been submitted to the thread pool's queue. int _num_unsubmitted_tasks; - - ThreadPoolToken(const ThreadPoolToken&) = delete; - void operator=(const ThreadPoolToken&) = delete; }; } // namespace doris diff --git a/be/src/util/work_thread_pool.hpp b/be/src/util/work_thread_pool.hpp index 00430ff75148fc..1da8a08f90d234 100644 --- a/be/src/util/work_thread_pool.hpp +++ b/be/src/util/work_thread_pool.hpp @@ -18,7 +18,6 @@ #pragma once #include -#include #include "util/blocking_priority_queue.hpp" #include "util/blocking_queue.hpp" @@ -126,12 +125,13 @@ class WorkThreadPool { } std::string get_info() const { - return fmt::format( - "PriorityThreadPool(name={}, queue_size={}/{}, active_thread={}/{}, " - "total_get_wait_time={}, total_put_wait_time={})", - _name, get_queue_size(), _work_queue.get_capacity(), _active_threads, - _threads.size(), _work_queue.total_get_wait_time(), - _work_queue.total_put_wait_time()); + return (Priority ? "PriorityThreadPool" : "FifoThreadPool") + + fmt::format( + "(name={}, queue_size={}/{}, active_thread={}/{}, " + "total_get_wait_time={}, total_put_wait_time={})", + _name, get_queue_size(), _work_queue.get_capacity(), _active_threads, + _threads.size(), _work_queue.total_get_wait_time(), + _work_queue.total_put_wait_time()); } protected: diff --git a/be/src/vec/aggregate_functions/aggregate_function_collect.cpp b/be/src/vec/aggregate_functions/aggregate_function_collect.cpp index d726b7c6355318..552d47d3567f9f 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_collect.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_collect.cpp @@ -72,12 +72,18 @@ AggregateFunctionPtr create_aggregate_function_collect_impl(const std::string& n if (which.is_date_or_datetime()) { return do_create_agg_function_collect(distinct, argument_types, result_is_nullable); - } else if (which.is_date_v2() || which.is_ipv4()) { + } else if (which.is_date_v2()) { return do_create_agg_function_collect(distinct, argument_types, result_is_nullable); - } else if (which.is_date_time_v2() || which.is_ipv6()) { + } else if (which.is_date_time_v2()) { return do_create_agg_function_collect(distinct, argument_types, result_is_nullable); + } else if (which.is_ipv6()) { + return do_create_agg_function_collect(distinct, argument_types, + result_is_nullable); + } else if (which.is_ipv4()) { + return do_create_agg_function_collect(distinct, argument_types, + result_is_nullable); } else if (which.is_string()) { return do_create_agg_function_collect( distinct, argument_types, result_is_nullable); diff --git a/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h index a81bdcddaa3310..d7d00140b497f7 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h +++ b/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h @@ -234,7 +234,7 @@ struct AggOrthBitmapExprCalBaseData { if (first_init) { DCHECK(argument_size > 1); const auto& col = - assert_cast(*columns[2]); + assert_cast(*columns[2]); std::string expr = col.get_data_at(row_num).to_string(); bitmap_expr_cal.bitmap_calculation_init(expr); first_init = false; diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index 4be4649fcadadc..e66e016381e83b 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -734,22 +734,17 @@ ColumnPtr ColumnArray::filter_generic(const Filter& filt, ssize_t result_size_hi if (size == 0) return ColumnArray::create(data); Filter nested_filt(get_offsets().back()); + ssize_t nested_result_size_hint = 0; for (size_t i = 0; i < size; ++i) { - if (filt[i]) + if (filt[i]) { memset(&nested_filt[offset_at(i)], 1, size_at(i)); - else + nested_result_size_hint += size_at(i); + } else { memset(&nested_filt[offset_at(i)], 0, size_at(i)); + } } auto res = ColumnArray::create(data->clone_empty()); - - ssize_t nested_result_size_hint = 0; - if (result_size_hint < 0) - nested_result_size_hint = result_size_hint; - else if (result_size_hint && result_size_hint < 1000000000 && - data->size() < 1000000000) /// Avoid overflow. - nested_result_size_hint = result_size_hint * data->size() / size; - res->data = data->filter(nested_filt, nested_result_size_hint); auto& res_offsets = res->get_offsets(); diff --git a/be/src/vec/common/allocator.cpp b/be/src/vec/common/allocator.cpp index d7bbb761c88cc7..1743952509d967 100644 --- a/be/src/vec/common/allocator.cpp +++ b/be/src/vec/common/allocator.cpp @@ -30,12 +30,10 @@ // Allocator is used by too many files. For compilation speed, put dependencies in `.cpp` as much as possible. #include "common/compiler_util.h" #include "common/status.h" -#include "runtime/fragment_mgr.h" #include "runtime/memory/global_memory_arbitrator.h" -#include "runtime/memory/mem_tracker_limiter.h" #include "runtime/memory/thread_mem_tracker_mgr.h" +#include "runtime/process_profile.h" #include "runtime/thread_context.h" -#include "util/defer_op.h" #include "util/mem_info.h" #include "util/stack_util.h" #include "util/uid_util.h" @@ -133,7 +131,7 @@ void Allocator::sys_mem if (wait_milliseconds >= doris::config::thread_wait_gc_max_milliseconds) { // Make sure to completely wait thread_wait_gc_max_milliseconds only once. doris::thread_context()->thread_mem_tracker_mgr->disable_wait_gc(); - doris::MemTrackerLimiter::print_log_process_usage(); + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); // If the external catch, throw bad::alloc first, let the query actively cancel. Otherwise asynchronous cancel. if (!doris::enable_thread_catch_bad_alloc) { LOG(INFO) << fmt::format( @@ -152,7 +150,6 @@ void Allocator::sys_mem // else, enough memory is available, the query continues execute. } else if (doris::enable_thread_catch_bad_alloc) { LOG(INFO) << fmt::format("sys memory check failed, throw exception, {}.", err_msg); - doris::MemTrackerLimiter::print_log_process_usage(); throw doris::Exception(doris::ErrorCode::MEM_ALLOC_FAILED, err_msg); } else { LOG(INFO) << fmt::format("sys memory check failed, no throw exception, {}.", err_msg); @@ -223,7 +220,7 @@ void Allocator::throw_b << fmt::format("{}, Stacktrace: {}", doris::GlobalMemoryArbitrator::process_mem_log_str(), doris::get_stack_trace()); - doris::MemTrackerLimiter::print_log_process_usage(); + doris::ProcessProfile::instance()->memory_profile()->print_log_process_usage(); throw doris::Exception(doris::ErrorCode::MEM_ALLOC_FAILED, err); } 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 13c40e60777258..2798596c82373b 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 @@ -101,6 +101,8 @@ class DataTypeArraySerDe : public DataTypeSerDe { nested_serde->set_return_object_as_string(value); } + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { return {nested_serde}; } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp b/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp index f52a3d1e9b4858..f60d054df31893 100644 --- a/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_bitmap_serde.cpp @@ -17,6 +17,7 @@ #include "data_type_bitmap_serde.h" +#include #include #include @@ -27,12 +28,36 @@ #include "vec/columns/column_const.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" +#include "vec/data_types/serde/data_type_nullable_serde.h" namespace doris { namespace vectorized { class IColumn; +Status DataTypeBitMapSerDe::serialize_column_to_json(const IColumn& column, int start_idx, + int end_idx, BufferWritable& bw, + FormatOptions& options) const { + SERIALIZE_COLUMN_TO_JSON(); +} + +Status DataTypeBitMapSerDe::serialize_one_cell_to_json(const IColumn& column, int row_num, + BufferWritable& bw, + FormatOptions& options) const { + /** + * For null values in ordinary types, we use \N to represent them; + * for null values in nested types, we use null to represent them, just like the json format. + */ + if (_nesting_level >= 2) { + bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), + strlen(NULL_IN_COMPLEX_TYPE.c_str())); + } else { + bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), + strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); + } + return Status::OK(); +} + Status DataTypeBitMapSerDe::deserialize_column_from_json_vector( IColumn& column, std::vector& slices, int* num_deserialized, const FormatOptions& options) const { @@ -95,6 +120,26 @@ void DataTypeBitMapSerDe::write_one_cell_to_jsonb(const IColumn& column, JsonbWr result.writeEndBinary(); } +void DataTypeBitMapSerDe::write_column_to_arrow(const IColumn& column, const NullMap* null_map, + arrow::ArrayBuilder* array_builder, int start, + int end, const cctz::time_zone& ctz) const { + const auto& col = assert_cast(column); + auto& builder = assert_cast(*array_builder); + for (size_t string_i = start; string_i < end; ++string_i) { + if (null_map && (*null_map)[string_i]) { + checkArrowStatus(builder.AppendNull(), column.get_name(), + array_builder->type()->name()); + } else { + auto& bitmap_value = const_cast(col.get_element(string_i)); + std::string memory_buffer(bitmap_value.getSizeInBytes(), '0'); + bitmap_value.write_to(memory_buffer.data()); + checkArrowStatus( + builder.Append(memory_buffer.data(), static_cast(memory_buffer.size())), + column.get_name(), array_builder->type()->name()); + } + } +} + void DataTypeBitMapSerDe::read_one_cell_from_jsonb(IColumn& column, const JsonbValue* arg) const { auto& col = reinterpret_cast(column); auto blob = static_cast(arg); @@ -147,11 +192,19 @@ Status DataTypeBitMapSerDe::write_column_to_orc(const std::string& timezone, con auto& col_data = assert_cast(column); orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + INIT_MEMORY_FOR_ORC_WRITER() + for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 1) { - const auto& ele = col_data.get_data_at(row_id); - cur_batch->data[row_id] = const_cast(ele.data); - cur_batch->length[row_id] = ele.size; + auto bitmap_value = const_cast(col_data.get_element(row_id)); + size_t len = bitmap_value.getSizeInBytes(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + bitmap_value.write_to(const_cast(bufferRef.data) + offset); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } 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 d4a643b3b163ae..ba7842e354c442 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 @@ -36,14 +36,10 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { DataTypeBitMapSerDe(int nesting_level = 1) : DataTypeSerDe(nesting_level) {}; Status serialize_one_cell_to_json(const IColumn& column, int row_num, BufferWritable& bw, - FormatOptions& options) const override { - return Status::NotSupported("serialize_one_cell_to_json with type [{}]", column.get_name()); - } + FormatOptions& options) const override; Status serialize_column_to_json(const IColumn& column, int start_idx, int end_idx, - BufferWritable& bw, FormatOptions& options) const override { - return Status::NotSupported("serialize_column_to_json with type [{}]", column.get_name()); - } + BufferWritable& bw, FormatOptions& options) const override; Status deserialize_one_cell_from_json(IColumn& column, Slice& slice, const FormatOptions& options) const override; @@ -63,10 +59,7 @@ class DataTypeBitMapSerDe : public DataTypeSerDe { void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int start, int end, - const cctz::time_zone& ctz) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_arrow with type " + column.get_name()); - } + const cctz::time_zone& ctz) const override; void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { 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 e749b2fa2e74a0..4cdd6b90326c03 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 @@ -288,16 +288,7 @@ Status DataTypeDate64SerDe::write_column_to_orc(const std::string& timezone, con auto& col_data = static_cast&>(column).get_data(); orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - if (!ptr) { - return Status::InternalError( - "malloc memory error when write largeint column data to orc file."); - } - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - const size_t begin_off = offset; + INIT_MEMORY_FOR_ORC_WRITER() for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 0) { @@ -309,18 +300,11 @@ Status DataTypeDate64SerDe::write_column_to_orc(const std::string& timezone, con REALLOC_MEMORY_FOR_ORC_WRITER() + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; cur_batch->length[row_id] = len; offset += len; } - size_t data_off = 0; - for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 1) { - cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; - data_off += cur_batch->length[row_id]; - } - } - buffer_list.emplace_back(bufferRef); cur_batch->numElements = end - start; return Status::OK(); } diff --git a/be/src/vec/data_types/serde/data_type_hll_serde.cpp b/be/src/vec/data_types/serde/data_type_hll_serde.cpp index 799e1c15d63b18..aba3a9d0619a71 100644 --- a/be/src/vec/data_types/serde/data_type_hll_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_hll_serde.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include "arrow/array/builder_binary.h" @@ -47,28 +48,17 @@ Status DataTypeHLLSerDe::serialize_column_to_json(const IColumn& column, int sta Status DataTypeHLLSerDe::serialize_one_cell_to_json(const IColumn& column, int row_num, BufferWritable& bw, FormatOptions& options) const { - if (!options._output_object_data) { - /** - * For null values in ordinary types, we use \N to represent them; - * for null values in nested types, we use null to represent them, just like the json format. - */ - if (_nesting_level >= 2) { - bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), - strlen(NULL_IN_COMPLEX_TYPE.c_str())); - } else { - bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), - strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); - } - return Status::OK(); + /** + * For null values in ordinary types, we use \N to represent them; + * for null values in nested types, we use null to represent them, just like the json format. + */ + if (_nesting_level >= 2) { + bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), + strlen(NULL_IN_COMPLEX_TYPE.c_str())); + } else { + bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), + strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); } - auto col_row = check_column_const_set_readability(column, row_num); - ColumnPtr ptr = col_row.first; - row_num = col_row.second; - auto& data = const_cast(assert_cast(*ptr).get_element(row_num)); - std::unique_ptr buf = - std::make_unique_for_overwrite(data.max_serialized_size()); - size_t size = data.serialize((uint8*)buf.get()); - bw.write(buf.get(), size); return Status::OK(); } @@ -137,7 +127,7 @@ void DataTypeHLLSerDe::write_column_to_arrow(const IColumn& column, const NullMa arrow::ArrayBuilder* array_builder, int start, int end, const cctz::time_zone& ctz) const { const auto& col = assert_cast(column); - auto& builder = assert_cast(*array_builder); + auto& builder = assert_cast(*array_builder); for (size_t string_i = start; string_i < end; ++string_i) { if (null_map && (*null_map)[string_i]) { checkArrowStatus(builder.AppendNull(), column.get_name(), @@ -195,11 +185,19 @@ Status DataTypeHLLSerDe::write_column_to_orc(const std::string& timezone, const auto& col_data = assert_cast(column); orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + INIT_MEMORY_FOR_ORC_WRITER() + for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 1) { - const auto& ele = col_data.get_data_at(row_id); - cur_batch->data[row_id] = const_cast(ele.data); - cur_batch->length[row_id] = ele.size; + auto hll_value = const_cast(col_data.get_element(row_id)); + size_t len = hll_value.max_serialized_size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + hll_value.serialize((uint8_t*)(bufferRef.data) + offset); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } diff --git a/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp b/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp index 612c9ce42227dd..468565ad2d2d9d 100644 --- a/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_ipv6_serde.cpp @@ -182,38 +182,23 @@ Status DataTypeIPv6SerDe::write_column_to_orc(const std::string& timezone, const int end, std::vector& buffer_list) const { const auto& col_data = assert_cast(column).get_data(); orc::StringVectorBatch* cur_batch = assert_cast(orc_col_batch); - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - if (!ptr) { - return Status::InternalError( - "malloc memory error when write largeint column data to orc file."); - } - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - const size_t begin_off = offset; - - for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 0) { - continue; - } - std::string ipv6_str = IPv6Value::to_string(col_data[row_id]); - size_t len = ipv6_str.size(); - REALLOC_MEMORY_FOR_ORC_WRITER() + INIT_MEMORY_FOR_ORC_WRITER() - strcpy(const_cast(bufferRef.data) + offset, ipv6_str.c_str()); - offset += len; - cur_batch->length[row_id] = len; - } - size_t data_off = 0; for (size_t row_id = start; row_id < end; row_id++) { if (cur_batch->notNull[row_id] == 1) { - cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; - data_off += cur_batch->length[row_id]; + std::string ipv6_str = IPv6Value::to_string(col_data[row_id]); + size_t len = ipv6_str.size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + strcpy(const_cast(bufferRef.data) + offset, ipv6_str.c_str()); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } - buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; return Status::OK(); } diff --git a/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp b/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp index 08514a6eea764a..eb6c783cf28579 100644 --- a/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_jsonb_serde.cpp @@ -21,6 +21,10 @@ #include #include +#include +#include +#include + #include "arrow/array/builder_binary.h" #include "common/exception.h" #include "common/status.h" @@ -136,7 +140,29 @@ Status DataTypeJsonbSerDe::write_column_to_orc(const std::string& timezone, cons const NullMap* null_map, orc::ColumnVectorBatch* orc_col_batch, int start, int end, std::vector& buffer_list) const { - return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); + auto* cur_batch = dynamic_cast(orc_col_batch); + const auto& string_column = assert_cast(column); + + INIT_MEMORY_FOR_ORC_WRITER() + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + std::string_view string_ref = string_column.get_data_at(row_id).to_string_view(); + auto serialized_value = std::make_unique( + JsonbToJson::jsonb_to_json_string(string_ref.data(), string_ref.size())); + auto len = serialized_value->size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + memcpy(const_cast(bufferRef.data) + offset, serialized_value->data(), len); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); } void convert_jsonb_to_rapidjson(const JsonbValue& val, rapidjson::Value& target, 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 5e10a7ec3f2a20..d95726824703ca 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 @@ -95,6 +95,10 @@ class DataTypeMapSerDe : public DataTypeSerDe { value_serde->set_return_object_as_string(value); } + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { + return {key_serde, value_serde}; + } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_nullable_serde.h b/be/src/vec/data_types/serde/data_type_nullable_serde.h index e9af344fb65f75..c7dac856621542 100644 --- a/be/src/vec/data_types/serde/data_type_nullable_serde.h +++ b/be/src/vec/data_types/serde/data_type_nullable_serde.h @@ -99,6 +99,8 @@ class DataTypeNullableSerDe : public DataTypeSerDe { int row_num) const override; Status read_one_cell_from_json(IColumn& column, const rapidjson::Value& result) const override; + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { return {nested_serde}; } + private: template Status _write_column_to_mysql(const IColumn& column, MysqlRowBuffer& result, diff --git a/be/src/vec/data_types/serde/data_type_number_serde.cpp b/be/src/vec/data_types/serde/data_type_number_serde.cpp index f4fb6bbbb1f9cf..972668e65fdd71 100644 --- a/be/src/vec/data_types/serde/data_type_number_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_number_serde.cpp @@ -339,38 +339,22 @@ Status DataTypeNumberSerDe::write_column_to_orc(const std::string& timezone, if constexpr (std::is_same_v) { // largeint orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); - char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); - if (!ptr) { - return Status::InternalError( - "malloc memory error when write largeint column data to orc file."); - } - StringRef bufferRef; - bufferRef.data = ptr; - bufferRef.size = BUFFER_UNIT_SIZE; - size_t offset = 0; - const size_t begin_off = offset; + INIT_MEMORY_FOR_ORC_WRITER() for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 0) { - continue; - } - std::string value_str = fmt::format("{}", col_data[row_id]); - size_t len = value_str.size(); + if (cur_batch->notNull[row_id] == 1) { + std::string value_str = fmt::format("{}", col_data[row_id]); + size_t len = value_str.size(); - REALLOC_MEMORY_FOR_ORC_WRITER() + REALLOC_MEMORY_FOR_ORC_WRITER() - strcpy(const_cast(bufferRef.data) + offset, value_str.c_str()); - offset += len; - cur_batch->length[row_id] = len; - } - size_t data_off = 0; - for (size_t row_id = start; row_id < end; row_id++) { - if (cur_batch->notNull[row_id] == 1) { - cur_batch->data[row_id] = const_cast(bufferRef.data) + begin_off + data_off; - data_off += cur_batch->length[row_id]; + strcpy(const_cast(bufferRef.data) + offset, value_str.c_str()); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; } } - buffer_list.emplace_back(bufferRef); + cur_batch->numElements = end - start; } else if constexpr (std::is_same_v || std::is_same_v) { // tinyint/boolean WRITE_INTEGRAL_COLUMN_TO_ORC(orc::ByteVectorBatch) diff --git a/be/src/vec/data_types/serde/data_type_object_serde.cpp b/be/src/vec/data_types/serde/data_type_object_serde.cpp index 383add39354fa2..1b4e1caf4ac671 100644 --- a/be/src/vec/data_types/serde/data_type_object_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_object_serde.cpp @@ -19,6 +19,9 @@ #include +#include +#include + #include "common/exception.h" #include "common/status.h" #include "vec/columns/column.h" @@ -192,6 +195,38 @@ Status DataTypeObjectSerDe::write_one_cell_to_json(const IColumn& column, rapidj return Status::OK(); } +Status DataTypeObjectSerDe::write_column_to_orc(const std::string& timezone, const IColumn& column, + const NullMap* null_map, + orc::ColumnVectorBatch* orc_col_batch, int start, + int end, + std::vector& buffer_list) const { + const auto* var = check_and_get_column(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + INIT_MEMORY_FOR_ORC_WRITER() + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + auto serialized_value = std::make_unique(); + if (!var->serialize_one_row_to_string(row_id, serialized_value.get())) { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Failed to serialize variant {}", + var->dump_structure()); + } + auto len = serialized_value->length(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + memcpy(const_cast(bufferRef.data) + offset, serialized_value->data(), len); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); +} + } // namespace vectorized } // namespace doris 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 314922f8694f52..41608dc3f85c03 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 @@ -89,9 +89,7 @@ class DataTypeObjectSerDe : public DataTypeSerDe { Status write_column_to_orc(const std::string& timezone, const IColumn& column, const NullMap* null_map, orc::ColumnVectorBatch* orc_col_batch, int start, int end, - std::vector& buffer_list) const override { - return Status::NotSupported("write_column_to_orc with type " + column.get_name()); - } + std::vector& buffer_list) const override; Status write_one_cell_to_json(const IColumn& column, rapidjson::Value& result, rapidjson::Document::AllocatorType& allocator, Arena& mem_pool, 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 e91fe0166e112b..e24a3a295439ee 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 @@ -17,6 +17,7 @@ #pragma once +#include #include #include #include @@ -32,6 +33,7 @@ #include "vec/columns/column_const.h" #include "vec/common/arena.h" #include "vec/common/string_ref.h" +#include "vec/data_types/serde/data_type_nullable_serde.h" namespace doris { @@ -43,12 +45,23 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { Status serialize_one_cell_to_json(const IColumn& column, int row_num, BufferWritable& bw, FormatOptions& options) const override { - return Status::NotSupported("serialize_one_cell_to_json with type [{}]", column.get_name()); + /** + * For null values in ordinary types, we use \N to represent them; + * for null values in nested types, we use null to represent them, just like the json format. + */ + if (_nesting_level >= 2) { + bw.write(DataTypeNullableSerDe::NULL_IN_COMPLEX_TYPE.c_str(), + strlen(NULL_IN_COMPLEX_TYPE.c_str())); + } else { + bw.write(DataTypeNullableSerDe::NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str(), + strlen(NULL_IN_CSV_FOR_ORDINARY_TYPE.c_str())); + } + return Status::OK(); } Status serialize_column_to_json(const IColumn& column, int start_idx, int end_idx, BufferWritable& bw, FormatOptions& options) const override { - return Status::NotSupported("serialize_column_to_json with type [{}]", column.get_name()); + SERIALIZE_COLUMN_TO_JSON(); } Status deserialize_one_cell_from_json(IColumn& column, Slice& slice, const FormatOptions& options) const override { @@ -102,8 +115,21 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { void write_column_to_arrow(const IColumn& column, const NullMap* null_map, arrow::ArrayBuilder* array_builder, int start, int end, const cctz::time_zone& ctz) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, - "write_column_to_arrow with type " + column.get_name()); + const auto& col = assert_cast(column); + auto& builder = assert_cast(*array_builder); + for (size_t string_i = start; string_i < end; ++string_i) { + if (null_map && (*null_map)[string_i]) { + checkArrowStatus(builder.AppendNull(), column.get_name(), + array_builder->type()->name()); + } else { + auto& quantile_state_value = const_cast(col.get_element(string_i)); + std::string memory_buffer(quantile_state_value.get_serialized_size(), '0'); + quantile_state_value.serialize((uint8_t*)memory_buffer.data()); + checkArrowStatus(builder.Append(memory_buffer.data(), + static_cast(memory_buffer.size())), + column.get_name(), array_builder->type()->name()); + } + } } void read_column_from_arrow(IColumn& column, const arrow::Array* arrow_array, int start, int end, const cctz::time_zone& ctz) const override { @@ -126,7 +152,27 @@ class DataTypeQuantileStateSerDe : public DataTypeSerDe { const NullMap* null_map, orc::ColumnVectorBatch* orc_col_batch, int start, int end, std::vector& buffer_list) const override { - return Status::NotSupported("write_column_to_orc with type [{}]", column.get_name()); + auto& col_data = assert_cast(column); + orc::StringVectorBatch* cur_batch = dynamic_cast(orc_col_batch); + + INIT_MEMORY_FOR_ORC_WRITER() + + for (size_t row_id = start; row_id < end; row_id++) { + if (cur_batch->notNull[row_id] == 1) { + auto quantilestate_value = const_cast(col_data.get_element(row_id)); + size_t len = quantilestate_value.get_serialized_size(); + + REALLOC_MEMORY_FOR_ORC_WRITER() + + quantilestate_value.serialize((uint8_t*)(bufferRef.data) + offset); + cur_batch->data[row_id] = const_cast(bufferRef.data) + offset; + cur_batch->length[row_id] = len; + offset += len; + } + } + + cur_batch->numElements = end - start; + return Status::OK(); } private: diff --git a/be/src/vec/data_types/serde/data_type_serde.h b/be/src/vec/data_types/serde/data_type_serde.h index 46236faa926c6f..105b1bbaedd9f3 100644 --- a/be/src/vec/data_types/serde/data_type_serde.h +++ b/be/src/vec/data_types/serde/data_type_serde.h @@ -75,6 +75,18 @@ struct ColumnVectorBatch; ++*num_deserialized; \ } +#define INIT_MEMORY_FOR_ORC_WRITER() \ + char* ptr = (char*)malloc(BUFFER_UNIT_SIZE); \ + if (!ptr) { \ + return Status::InternalError( \ + "malloc memory error when write largeint column data to orc file."); \ + } \ + StringRef bufferRef; \ + bufferRef.data = ptr; \ + bufferRef.size = BUFFER_UNIT_SIZE; \ + size_t offset = 0; \ + buffer_list.emplace_back(bufferRef); + #define REALLOC_MEMORY_FOR_ORC_WRITER() \ while (bufferRef.size - BUFFER_RESERVED_SIZE < offset + len) { \ char* new_ptr = (char*)malloc(bufferRef.size + BUFFER_UNIT_SIZE); \ @@ -98,6 +110,10 @@ class IColumn; class Arena; class IDataType; +class DataTypeSerDe; +using DataTypeSerDeSPtr = std::shared_ptr; +using DataTypeSerDeSPtrs = std::vector; + // Deserialize means read from different file format or memory format, // for example read from arrow, read from parquet. // Serialize means write the column cell or the total column into another @@ -332,6 +348,11 @@ class DataTypeSerDe { Arena& mem_pool, int row_num) const; virtual Status read_one_cell_from_json(IColumn& column, const rapidjson::Value& result) const; + virtual DataTypeSerDeSPtrs get_nested_serdes() const { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method get_nested_serdes is not supported for this serde"); + } + protected: bool _return_object_as_string = false; // This parameter indicates what level the serde belongs to and is mainly used for complex types @@ -374,9 +395,6 @@ inline void checkArrowStatus(const arrow::Status& status, const std::string& col } } -using DataTypeSerDeSPtr = std::shared_ptr; -using DataTypeSerDeSPtrs = std::vector; - DataTypeSerDeSPtrs create_data_type_serdes( const std::vector>& types); DataTypeSerDeSPtrs create_data_type_serdes(const std::vector& slots); 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 84e988e150bbdc..5cd6f89e42fd6e 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 @@ -171,6 +171,8 @@ class DataTypeStructSerDe : public DataTypeSerDe { } } + virtual DataTypeSerDeSPtrs get_nested_serdes() const override { return elem_serdes_ptrs; } + private: std::optional try_get_position_by_name(const String& name) const; diff --git a/be/src/vec/exec/format/json/new_json_reader.cpp b/be/src/vec/exec/format/json/new_json_reader.cpp index e3c2c1f332e11b..307edc265beb6d 100644 --- a/be/src/vec/exec/format/json/new_json_reader.cpp +++ b/be/src/vec/exec/format/json/new_json_reader.cpp @@ -54,8 +54,11 @@ #include "util/slice.h" #include "util/uid_util.h" #include "vec/columns/column.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_map.h" #include "vec/columns/column_nullable.h" #include "vec/columns/column_string.h" +#include "vec/columns/column_struct.h" #include "vec/common/assert_cast.h" #include "vec/common/typeid_cast.h" #include "vec/core/block.h" @@ -164,10 +167,18 @@ void NewJsonReader::_init_file_description() { } Status NewJsonReader::init_reader( - const std::unordered_map& col_default_value_ctx) { + const std::unordered_map& col_default_value_ctx, + bool is_load) { + _is_load = is_load; + // generate _col_default_value_map RETURN_IF_ERROR(_get_column_default_value(_file_slot_descs, col_default_value_ctx)); + //use serde insert data to column. + for (auto* slot_desc : _file_slot_descs) { + _serdes.emplace_back(slot_desc->get_data_type_ptr()->get_serde()); + } + // create decompressor. // _decompressor may be nullptr if this is not a compressed file RETURN_IF_ERROR(Decompressor::create_decompressor(_file_compress_type, &_decompressor)); @@ -387,6 +398,9 @@ Status NewJsonReader::_get_range_params() { if (_params.file_attributes.__isset.fuzzy_parse) { _fuzzy_parse = _params.file_attributes.fuzzy_parse; } + if (_range.table_format_params.table_format_type == "hive") { + _is_hive_table = true; + } return Status::OK(); } @@ -474,8 +488,8 @@ Status NewJsonReader::_vhandle_simple_json(RuntimeState* /*state*/, Block& block bool valid = false; if (_next_row >= _total_rows) { // parse json and generic document Status st = _parse_json(is_empty_row, eof); - if (st.is()) { - continue; // continue to read next + if (_is_load && st.is()) { + continue; // continue to read next (for load, after this , already append error to file.) } RETURN_IF_ERROR(st); if (*is_empty_row) { @@ -752,7 +766,21 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl int ctx_idx = 0; bool has_valid_value = false; - for (auto* slot_desc : slot_descs) { + + if (_is_hive_table) { + //don't like _fuzzy_parse,each line read in must modify name_map once. + + for (int i = 0; i < objectValue.MemberCount(); ++i) { + auto it = objectValue.MemberBegin() + i; + std::string field_name(it->name.GetString(), it->name.GetStringLength()); + std::transform(field_name.begin(), field_name.end(), field_name.begin(), ::tolower); + + //Use the last value with the same name. + _name_map.emplace(field_name, i); + } + } + for (size_t slot_idx = 0; slot_idx < slot_descs.size(); ++slot_idx) { + auto* slot_desc = slot_descs[slot_idx]; if (!slot_desc->is_materialized()) { continue; } @@ -761,7 +789,7 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl auto* column_ptr = block.get_by_position(dest_index).column->assume_mutable().get(); rapidjson::Value::ConstMemberIterator it = objectValue.MemberEnd(); - if (_fuzzy_parse) { + if (_fuzzy_parse || _is_hive_table) { auto idx_it = _name_map.find(slot_desc->col_name()); if (idx_it != _name_map.end() && idx_it->second < objectValue.MemberCount()) { it = objectValue.MemberBegin() + idx_it->second; @@ -773,20 +801,21 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl if (it != objectValue.MemberEnd()) { const rapidjson::Value& value = it->value; - RETURN_IF_ERROR(_write_data_to_column(&value, slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_write_data_to_column(&value, slot_desc->type(), column_ptr, + slot_desc->col_name(), _serdes[slot_idx], valid)); if (!(*valid)) { return Status::OK(); } has_valid_value = true; } else { // not found, filling with default value - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[slot_idx], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } } } - if (!has_valid_value) { + if (!has_valid_value && _is_load) { // there is no valid value in json line but has filled with default value before // so remove this line in block string col_names; @@ -810,79 +839,188 @@ Status NewJsonReader::_set_column_value(rapidjson::Value& objectValue, Block& bl } Status NewJsonReader::_write_data_to_column(rapidjson::Value::ConstValueIterator value, - SlotDescriptor* slot_desc, IColumn* column_ptr, + const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, + const std::string& column_name, DataTypeSerDeSPtr serde, bool* valid) { - const char* str_value = nullptr; - char tmp_buf[128] = {0}; - int32_t wbytes = 0; - std::string json_str; - ColumnNullable* nullable_column = nullptr; - if (slot_desc->is_nullable()) { + vectorized::IColumn* data_column_ptr = column_ptr; + DataTypeSerDeSPtr data_serde = serde; + + bool value_is_null = (value == nullptr) || (value->GetType() == rapidjson::Type::kNullType); + + if (column_ptr->is_nullable()) { nullable_column = reinterpret_cast(column_ptr); - // kNullType will put 1 into the Null map, so there is no need to push 0 for kNullType. - if (value->GetType() != rapidjson::Type::kNullType) { + data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_serde = serde->get_nested_serdes()[0]; + + if (value_is_null) { + nullable_column->insert_default(); + *valid = true; + return Status::OK(); + } else { nullable_column->get_null_map_data().push_back(0); + } + + } else if (value_is_null) [[unlikely]] { + if (_is_load) { + RETURN_IF_ERROR(_append_error_msg( + *value, "Json value is null, but the column `{}` is not nullable.", column_name, + valid)); + return Status::OK(); + } else { - nullable_column->insert_default(); + return Status::DataQualityError( + "Json value is null, but the column `{}` is not nullable.", column_name); } - column_ptr = &nullable_column->get_nested_column(); } - switch (value->GetType()) { - case rapidjson::Type::kStringType: - str_value = value->GetString(); - wbytes = value->GetStringLength(); - break; - case rapidjson::Type::kNumberType: - if (value->IsUint()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%u", value->GetUint()); - } else if (value->IsInt()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%d", value->GetInt()); - } else if (value->IsUint64()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRIu64, value->GetUint64()); - } else if (value->IsInt64()) { - wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRId64, value->GetInt64()); - } else if (value->IsFloat() || value->IsDouble()) { - auto* end = fmt::format_to(tmp_buf, "{}", value->GetDouble()); - wbytes = end - tmp_buf; + if (_is_load || !type_desc.is_complex_type()) { + if (value->IsString()) { + Slice slice {value->GetString(), value->GetStringLength()}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); + } else { - return Status::InternalError("It should not here."); + // We can `switch (value->GetType()) case: kNumberType`. + // Note that `if (value->IsInt())`, but column is FloatColumn. + // Or for any type, use `NewJsonReader::_print_json_value(*value)`. + + const char* str_value = nullptr; + char tmp_buf[128] = {0}; + size_t wbytes = 0; + std::string json_str; + + switch (value->GetType()) { + case rapidjson::Type::kStringType: + str_value = value->GetString(); + wbytes = value->GetStringLength(); + break; + case rapidjson::Type::kNumberType: + if (value->IsUint()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%u", value->GetUint()); + } else if (value->IsInt()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%d", value->GetInt()); + } else if (value->IsUint64()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRIu64, value->GetUint64()); + } else if (value->IsInt64()) { + wbytes = snprintf(tmp_buf, sizeof(tmp_buf), "%" PRId64, value->GetInt64()); + } else if (value->IsFloat() || value->IsDouble()) { + auto* end = fmt::format_to(tmp_buf, "{}", value->GetDouble()); + wbytes = end - tmp_buf; + } else { + return Status::InternalError("It should not here."); + } + str_value = tmp_buf; + break; + case rapidjson::Type::kFalseType: + wbytes = 1; + str_value = (char*)"0"; + break; + case rapidjson::Type::kTrueType: + wbytes = 1; + str_value = (char*)"1"; + break; + default: + // for other type, we convert it to string to save + json_str = NewJsonReader::_print_json_value(*value); + wbytes = json_str.size(); + str_value = json_str.c_str(); + break; + } + Slice slice {str_value, wbytes}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); } - str_value = tmp_buf; - break; - case rapidjson::Type::kFalseType: - wbytes = 1; - str_value = (char*)"0"; - break; - case rapidjson::Type::kTrueType: - wbytes = 1; - str_value = (char*)"1"; - break; - case rapidjson::Type::kNullType: - if (!slot_desc->is_nullable()) { - RETURN_IF_ERROR(_append_error_msg( - *value, "Json value is null, but the column `{}` is not nullable.", - slot_desc->col_name(), valid)); - return Status::OK(); + } else if (type_desc.type == TYPE_STRUCT) { + if (!value->IsObject()) [[unlikely]] { + return Status::DataQualityError( + "Json value isn't object, but the column `{}` is struct.", column_name); } - // return immediately to prevent from repeatedly insert_data - *valid = true; - return Status::OK(); - default: - // for other type like array or object. we convert it to string to save - json_str = NewJsonReader::_print_json_value(*value); - wbytes = json_str.size(); - str_value = json_str.c_str(); - break; - } + auto sub_col_size = type_desc.children.size(); + const auto& struct_value = value->GetObject(); + + auto sub_serdes = data_serde->get_nested_serdes(); + auto struct_column_ptr = assert_cast(data_column_ptr); + + std::map sub_col_name_to_idx; + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + sub_col_name_to_idx.emplace(type_desc.field_names[sub_col_idx], sub_col_idx); + } + + std::vector sub_values(sub_col_size, nullptr); + for (const auto& sub : struct_value) { + if (!sub.name.IsString()) [[unlikely]] { + return Status::DataQualityError( + "Json file struct column `{}` subfield name isn't a String", column_name); + } + + auto sub_key_char = sub.name.GetString(); + auto sub_key_length = sub.name.GetStringLength(); + + std::string sub_key(sub_key_char, sub_key_length); + std::transform(sub_key.begin(), sub_key.end(), sub_key.begin(), ::tolower); + + if (sub_col_name_to_idx.find(sub_key) == sub_col_name_to_idx.end()) [[unlikely]] { + continue; + } + size_t sub_column_idx = sub_col_name_to_idx[sub_key]; + sub_values[sub_column_idx] = &sub.value; + } + + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + auto sub_value = sub_values[sub_col_idx]; + + const auto& sub_col_type = type_desc.children[sub_col_idx]; + + RETURN_IF_ERROR(_write_data_to_column( + sub_value, sub_col_type, struct_column_ptr->get_column(sub_col_idx).get_ptr(), + column_name + "." + type_desc.field_names[sub_col_idx], sub_serdes[sub_col_idx], + valid)); + } + } else if (type_desc.type == TYPE_MAP) { + if (!value->IsObject()) [[unlikely]] { + return Status::DataQualityError("Json value isn't object, but the column `{}` is map.", + column_name); + } + const auto& object_value = value->GetObject(); + auto sub_serdes = data_serde->get_nested_serdes(); + auto map_column_ptr = assert_cast(data_column_ptr); - // TODO: if the vexpr can support another 'slot_desc type' than 'TYPE_VARCHAR', - // we need use a function to support these types to insert data in columns. - DCHECK(slot_desc->type().type == TYPE_VARCHAR || slot_desc->type().type == TYPE_STRING) - << slot_desc->type().type << ", query id: " << print_id(_state->query_id()); - assert_cast(column_ptr)->insert_data(str_value, wbytes); + for (const auto& member_value : object_value) { + RETURN_IF_ERROR(_write_data_to_column( + &member_value.name, type_desc.children[0], + map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr(), + column_name + ".key", sub_serdes[0], valid)); + + RETURN_IF_ERROR(_write_data_to_column( + &member_value.value, type_desc.children[1], + map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr(), + column_name + ".value", sub_serdes[1], valid)); + } + + auto& offsets = map_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + object_value.MemberCount()); + } else if (type_desc.type == TYPE_ARRAY) { + if (!value->IsArray()) [[unlikely]] { + return Status::DataQualityError("Json value isn't array, but the column `{}` is array.", + column_name); + } + const auto& array_value = value->GetArray(); + auto sub_serdes = data_serde->get_nested_serdes(); + auto array_column_ptr = assert_cast(data_column_ptr); + + for (const auto& sub_value : array_value) { + RETURN_IF_ERROR(_write_data_to_column(&sub_value, type_desc.children[0], + array_column_ptr->get_data().get_ptr(), + column_name + ".element", sub_serdes[0], valid)); + } + auto& offsets = array_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + array_value.Size()); + } else { + return Status::InternalError("Not support load to complex column."); + } *valid = true; return Status::OK(); @@ -914,20 +1052,21 @@ Status NewJsonReader::_write_columns_by_jsonpath(rapidjson::Value& objectValue, // if json_values' size > 1, it means we just match an array, not a wrapped one, so no need to unwrap. json_values = &((*json_values)[0]); } - RETURN_IF_ERROR(_write_data_to_column(json_values, slot_descs[i], column_ptr, valid)); + RETURN_IF_ERROR(_write_data_to_column(json_values, slot_descs[i]->type(), column_ptr, + slot_descs[i]->col_name(), _serdes[i], valid)); if (!(*valid)) { return Status::OK(); } has_valid_value = true; } else { // not found, filling with default value - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[i], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } } } - if (!has_valid_value) { + if (!has_valid_value && _is_load) { // there is no valid value in json line but has filled with default value before // so remove this line in block for (int i = 0; i < block.columns(); ++i) { @@ -1074,7 +1213,7 @@ Status NewJsonReader::_simdjson_handle_simple_json(RuntimeState* /*state*/, Bloc // step2: get json value by json doc Status st = _get_json_value(&size, eof, &error, is_empty_row); - if (st.is()) { + if (_is_load && st.is()) { return Status::OK(); } RETURN_IF_ERROR(st); @@ -1349,25 +1488,39 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val for (auto field : *value) { std::string_view key = field.unescaped_key(); StringRef name_ref(key.data(), key.size()); + std::string key_string; + if (_is_hive_table) { + key_string = name_ref.to_string(); + std::transform(key_string.begin(), key_string.end(), key_string.begin(), ::tolower); + name_ref = StringRef(key_string); + } const size_t column_index = _column_index(name_ref, key_index++); if (UNLIKELY(ssize_t(column_index) < 0)) { // This key is not exist in slot desc, just ignore continue; } if (_seen_columns[column_index]) { - continue; + if (_is_hive_table) { + //Since value can only be traversed once, + // we can only insert the original value first, then delete it, and then reinsert the new value + block.get_by_position(column_index).column->assume_mutable()->pop_back(1); + } else { + continue; + } } simdjson::ondemand::value val = field.value(); auto* column_ptr = block.get_by_position(column_index).column->assume_mutable().get(); - RETURN_IF_ERROR( - _simdjson_write_data_to_column(val, slot_descs[column_index], column_ptr, valid)); + RETURN_IF_ERROR(_simdjson_write_data_to_column( + val, slot_descs[column_index]->type(), column_ptr, + slot_descs[column_index]->col_name(), _serdes[column_index], valid)); if (!(*valid)) { return Status::OK(); } _seen_columns[column_index] = true; has_valid_value = true; } - if (!has_valid_value) { + + if (!has_valid_value && _is_load) { string col_names; for (auto* slot_desc : slot_descs) { col_names.append(slot_desc->col_name() + ", "); @@ -1400,7 +1553,7 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val auto* column_ptr = block.get_by_position(i).column->assume_mutable().get(); if (column_ptr->size() < cur_row_count + 1) { DCHECK(column_ptr->size() == cur_row_count); - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[i], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } @@ -1409,12 +1562,6 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val DCHECK(column_ptr->size() == cur_row_count + 1); } -#ifndef NDEBUG - // Check all columns rows matched - for (size_t i = 0; i < block.columns(); ++i) { - DCHECK_EQ(block.get_by_position(i).column->size(), cur_row_count + 1); - } -#endif // There is at least one valid value here DCHECK(nullcount < block.columns()); *valid = true; @@ -1422,54 +1569,180 @@ Status NewJsonReader::_simdjson_set_column_value(simdjson::ondemand::object* val } Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& value, - SlotDescriptor* slot_desc, IColumn* column, - bool* valid) { - // write + const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, + const std::string& column_name, + DataTypeSerDeSPtr serde, bool* valid) { ColumnNullable* nullable_column = nullptr; - IColumn* column_ptr = nullptr; - if (slot_desc->is_nullable()) { - nullable_column = assert_cast(column); - column_ptr = &nullable_column->get_nested_column(); - } - // TODO: if the vexpr can support another 'slot_desc type' than 'TYPE_VARCHAR', - // we need use a function to support these types to insert data in columns. - auto* column_string = assert_cast(column_ptr); - switch (value.type()) { - case simdjson::ondemand::json_type::null: { - if (column->is_nullable()) { - // insert_default already push 1 to null_map - nullable_column->insert_default(); + vectorized::IColumn* data_column_ptr = column_ptr; + DataTypeSerDeSPtr data_serde = serde; + + if (column_ptr->is_nullable()) { + nullable_column = reinterpret_cast(column_ptr); + + data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_serde = serde->get_nested_serdes()[0]; + + // kNullType will put 1 into the Null map, so there is no need to push 0 for kNullType. + if (value.type() != simdjson::ondemand::json_type::null) { + nullable_column->get_null_map_data().push_back(0); } else { + nullable_column->insert_default(); + *valid = true; + return Status::OK(); + } + } else if (value.type() == simdjson::ondemand::json_type::null) [[unlikely]] { + if (_is_load) { RETURN_IF_ERROR(_append_error_msg( nullptr, "Json value is null, but the column `{}` is not nullable.", - slot_desc->col_name(), valid)); + column_name, valid)); return Status::OK(); - } - break; - } - case simdjson::ondemand::json_type::boolean: { - nullable_column->get_null_map_data().push_back(0); - if (value.get_bool()) { - column_string->insert_data("1", 1); } else { - column_string->insert_data("0", 1); + return Status::DataQualityError( + "Json value is null, but the column `{}` is not nullable.", column_name); } - break; } - default: { + + if (_is_load || !type_desc.is_complex_type()) { if (value.type() == simdjson::ondemand::json_type::string) { - auto* unescape_buffer = - reinterpret_cast(_simdjson_ondemand_unscape_padding_buffer.data()); - std::string_view unescaped_value = - _ondemand_json_parser->unescape(value.get_raw_json_string(), unescape_buffer); - nullable_column->get_null_map_data().push_back(0); - column_string->insert_data(unescaped_value.data(), unescaped_value.length()); - break; + std::string_view value_string = value.get_string(); + Slice slice {value_string.data(), value_string.size()}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); + + } else { + // Maybe we can `switch (value->GetType()) case: kNumberType`. + // Note that `if (value->IsInt())`, but column is FloatColumn. + std::string_view json_str = simdjson::to_json_string(value); + Slice slice {json_str.data(), json_str.size()}; + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + _serde_options)); + } + } else if (type_desc.type == TYPE_STRUCT) { + if (value.type() != simdjson::ondemand::json_type::object) [[unlikely]] { + return Status::DataQualityError( + "Json value isn't object, but the column `{}` is struct.", column_name); + } + + auto sub_col_size = type_desc.children.size(); + simdjson::ondemand::object struct_value = value.get_object(); + auto sub_serdes = data_serde->get_nested_serdes(); + auto struct_column_ptr = assert_cast(data_column_ptr); + + std::map sub_col_name_to_idx; + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + sub_col_name_to_idx.emplace(type_desc.field_names[sub_col_idx], sub_col_idx); + } + vector has_value(sub_col_size, false); + for (simdjson::ondemand::field sub : struct_value) { + std::string_view sub_key_view = sub.unescaped_key(); + std::string sub_key(sub_key_view.data(), sub_key_view.length()); + std::transform(sub_key.begin(), sub_key.end(), sub_key.begin(), ::tolower); + + if (sub_col_name_to_idx.find(sub_key) == sub_col_name_to_idx.end()) [[unlikely]] { + continue; + } + size_t sub_column_idx = sub_col_name_to_idx[sub_key]; + auto sub_column_ptr = struct_column_ptr->get_column(sub_column_idx).get_ptr(); + + if (has_value[sub_column_idx]) [[unlikely]] { + // Since struct_value can only be traversed once, we can only insert + // the original value first, then delete it, and then reinsert the new value. + sub_column_ptr->pop_back(1); + } + has_value[sub_column_idx] = true; + + const auto& sub_col_type = type_desc.children[sub_column_idx]; + RETURN_IF_ERROR(_simdjson_write_data_to_column( + sub.value(), sub_col_type, sub_column_ptr, column_name + "." + sub_key, + sub_serdes[sub_column_idx], valid)); } - auto value_str = simdjson::to_json_string(value).value(); - nullable_column->get_null_map_data().push_back(0); - column_string->insert_data(value_str.data(), value_str.length()); - } + + //fill missing subcolumn + for (size_t sub_col_idx = 0; sub_col_idx < sub_col_size; sub_col_idx++) { + if (has_value[sub_col_idx] == true) { + continue; + } + + auto sub_column_ptr = struct_column_ptr->get_column(sub_col_idx).get_ptr(); + if (sub_column_ptr->is_nullable()) { + sub_column_ptr->insert_default(); + continue; + } else [[unlikely]] { + return Status::DataQualityError( + "Json file structColumn miss field {} and this column isn't nullable.", + column_name + "." + type_desc.field_names[sub_col_idx]); + } + } + } else if (type_desc.type == TYPE_MAP) { + if (value.type() != simdjson::ondemand::json_type::object) [[unlikely]] { + return Status::DataQualityError("Json value isn't object, but the column `{}` is map.", + column_name); + } + simdjson::ondemand::object object_value = value.get_object(); + + auto sub_serdes = data_serde->get_nested_serdes(); + auto map_column_ptr = assert_cast(data_column_ptr); + + size_t field_count = 0; + for (simdjson::ondemand::field member_value : object_value) { + auto f = [](std::string_view key_view, const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, DataTypeSerDeSPtr serde, + vectorized::DataTypeSerDe::FormatOptions serde_options, bool* valid) { + auto data_column_ptr = column_ptr; + auto data_serde = serde; + if (column_ptr->is_nullable()) { + auto nullable_column = static_cast(column_ptr); + + nullable_column->get_null_map_data().push_back(0); + data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_serde = serde->get_nested_serdes()[0]; + } + Slice slice(key_view.data(), key_view.length()); + + RETURN_IF_ERROR(data_serde->deserialize_one_cell_from_json(*data_column_ptr, slice, + serde_options)); + return Status::OK(); + }; + + RETURN_IF_ERROR(f(member_value.unescaped_key(), type_desc.children[0], + map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr(), + sub_serdes[0], _serde_options, valid)); + + simdjson::ondemand::value field_value = member_value.value(); + RETURN_IF_ERROR(_simdjson_write_data_to_column( + field_value, type_desc.children[1], + map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr(), + column_name + ".value", sub_serdes[1], valid)); + field_count++; + } + + auto& offsets = map_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + field_count); + + } else if (type_desc.type == TYPE_ARRAY) { + if (value.type() != simdjson::ondemand::json_type::array) [[unlikely]] { + return Status::DataQualityError("Json value isn't array, but the column `{}` is array.", + column_name); + } + + simdjson::ondemand::array array_value = value.get_array(); + + auto sub_serdes = data_serde->get_nested_serdes(); + auto array_column_ptr = assert_cast(data_column_ptr); + + int field_count = 0; + for (simdjson::ondemand::value sub_value : array_value) { + RETURN_IF_ERROR(_simdjson_write_data_to_column( + sub_value, type_desc.children[0], array_column_ptr->get_data().get_ptr(), + column_name + ".element", sub_serdes[0], valid)); + field_count++; + } + auto& offsets = array_column_ptr->get_offsets(); + offsets.emplace_back(offsets.back() + field_count); + + } else { + return Status::InternalError("Not support load to complex column."); } *valid = true; return Status::OK(); @@ -1677,13 +1950,14 @@ Status NewJsonReader::_simdjson_write_columns_by_jsonpath( has_valid_value = true; } else if (i >= _parsed_jsonpaths.size() || st.is()) { // not match in jsondata, filling with default value - RETURN_IF_ERROR(_fill_missing_column(slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_fill_missing_column(slot_desc, _serdes[i], column_ptr, valid)); if (!(*valid)) { return Status::OK(); } } else { - RETURN_IF_ERROR( - _simdjson_write_data_to_column(json_value, slot_desc, column_ptr, valid)); + RETURN_IF_ERROR(_simdjson_write_data_to_column(json_value, slot_desc->type(), + column_ptr, slot_desc->col_name(), + _serdes[i], valid)); if (!(*valid)) { return Status::OK(); } @@ -1741,25 +2015,30 @@ Status NewJsonReader::_get_column_default_value( return Status::OK(); } -Status NewJsonReader::_fill_missing_column(SlotDescriptor* slot_desc, IColumn* column_ptr, - bool* valid) { - if (slot_desc->is_nullable()) { - auto* nullable_column = reinterpret_cast(column_ptr); - column_ptr = &nullable_column->get_nested_column(); - auto col_value = _col_default_value_map.find(slot_desc->col_name()); - if (col_value == _col_default_value_map.end()) { +Status NewJsonReader::_fill_missing_column(SlotDescriptor* slot_desc, DataTypeSerDeSPtr serde, + IColumn* column_ptr, bool* valid) { + auto col_value = _col_default_value_map.find(slot_desc->col_name()); + if (col_value == _col_default_value_map.end()) { + if (slot_desc->is_nullable()) { + auto* nullable_column = static_cast(column_ptr); nullable_column->insert_default(); } else { - const std::string& v_str = col_value->second; - nullable_column->get_null_map_data().push_back(0); - assert_cast(column_ptr)->insert_data(v_str.c_str(), v_str.size()); + if (_is_load) { + RETURN_IF_ERROR(_append_error_msg( + nullptr, "The column `{}` is not nullable, but it's not found in jsondata.", + slot_desc->col_name(), valid)); + } else { + return Status::DataQualityError( + "The column `{}` is not nullable, but it's not found in jsondata.", + slot_desc->col_name()); + } } } else { - RETURN_IF_ERROR(_append_error_msg( - nullptr, "The column `{}` is not nullable, but it's not found in jsondata.", - slot_desc->col_name(), valid)); + const std::string& v_str = col_value->second; + Slice column_default_value {v_str}; + RETURN_IF_ERROR(serde->deserialize_one_cell_from_json(*column_ptr, column_default_value, + _serde_options)); } - *valid = true; return Status::OK(); } diff --git a/be/src/vec/exec/format/json/new_json_reader.h b/be/src/vec/exec/format/json/new_json_reader.h index 0df3747b8c2a38..6828b6b2abfadb 100644 --- a/be/src/vec/exec/format/json/new_json_reader.h +++ b/be/src/vec/exec/format/json/new_json_reader.h @@ -88,7 +88,8 @@ class NewJsonReader : public GenericReader { ~NewJsonReader() override = default; Status init_reader(const std::unordered_map& - col_default_value_ctx); + col_default_value_ctx, + bool is_load); Status get_next_block(Block* block, size_t* read_rows, bool* eof) override; Status get_columns(std::unordered_map* name_to_type, std::unordered_set* missing_cols) override; @@ -129,7 +130,8 @@ class NewJsonReader : public GenericReader { const std::vector& slot_descs, bool* valid); Status _write_data_to_column(rapidjson::Value::ConstValueIterator value, - SlotDescriptor* slot_desc, vectorized::IColumn* column_ptr, + const TypeDescriptor& type_desc, vectorized::IColumn* column_ptr, + const std::string& column_name, DataTypeSerDeSPtr serde, bool* valid); Status _write_columns_by_jsonpath(rapidjson::Value& objectValue, @@ -178,8 +180,10 @@ class NewJsonReader : public GenericReader { const std::vector& slot_descs, bool* valid); Status _simdjson_write_data_to_column(simdjson::ondemand::value& value, - SlotDescriptor* slot_desc, - vectorized::IColumn* column_ptr, bool* valid); + const TypeDescriptor& type_desc, + vectorized::IColumn* column_ptr, + const std::string& column_name, DataTypeSerDeSPtr serde, + bool* valid); Status _simdjson_write_columns_by_jsonpath(simdjson::ondemand::object* value, const std::vector& slot_descs, @@ -197,8 +201,8 @@ class NewJsonReader : public GenericReader { const std::unordered_map& col_default_value_ctx); - Status _fill_missing_column(SlotDescriptor* slot_desc, vectorized::IColumn* column_ptr, - bool* valid); + Status _fill_missing_column(SlotDescriptor* slot_desc, DataTypeSerDeSPtr serde, + vectorized::IColumn* column_ptr, bool* valid); RuntimeState* _state = nullptr; RuntimeProfile* _profile = nullptr; @@ -283,6 +287,22 @@ class NewJsonReader : public GenericReader { std::unique_ptr _ondemand_json_parser; // column to default value string map std::unordered_map _col_default_value_map; + + bool _is_load = true; + //Used to indicate whether it is a stream load. When loading, only data will be inserted into columnString. + //If an illegal value is encountered during the load process, `_append_error_msg` should be called + //instead of directly returning `Status::DataQualityError` + + bool _is_hive_table = false; + // In hive : create table xxx ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; + // Hive will not allow you to create columns with the same name but different case, including field names inside + // structs, and will automatically convert uppercase names in create sql to lowercase.However, when Hive loads data + // to table, the column names in the data may be uppercase,and there may be multiple columns with + // the same name but different capitalization.We refer to the behavior of hive, convert all column names + // in the data to lowercase,and use the last one as the insertion value + + DataTypeSerDeSPtrs _serdes; + vectorized::DataTypeSerDe::FormatOptions _serde_options; }; } // namespace vectorized diff --git a/be/src/vec/exec/format/parquet/schema_desc.cpp b/be/src/vec/exec/format/parquet/schema_desc.cpp index de879706264028..1eae65b1a4db19 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.cpp +++ b/be/src/vec/exec/format/parquet/schema_desc.cpp @@ -137,6 +137,9 @@ Status FieldDescriptor::parse_from_thrift(const std::vectorsecond.data()}; +} + Status FieldDescriptor::parse_node_field(const std::vector& t_schemas, size_t curr_pos, FieldSchema* node_field) { if (curr_pos >= t_schemas.size()) { @@ -172,6 +183,7 @@ Status FieldDescriptor::parse_node_field(const std::vectortype.add_sub_type(child->type); node_field->is_nullable = false; _next_schema_pos = curr_pos + 1; + node_field->field_id = t_schema.__isset.field_id ? t_schema.field_id : -1; } else { bool is_optional = is_optional_node(t_schema); if (is_optional) { @@ -194,6 +206,7 @@ void FieldDescriptor::parse_physical_field(const tparquet::SchemaElement& physic auto type = get_doris_type(physical_schema); physical_field->type = type.first; physical_field->is_type_compatibility = type.second; + physical_field->field_id = physical_schema.__isset.field_id ? physical_schema.field_id : -1; } std::pair FieldDescriptor::get_doris_type( @@ -465,6 +478,7 @@ Status FieldDescriptor::parse_group_field(const std::vectortype.type = TYPE_ARRAY; group_field->type.add_sub_type(struct_field->type); group_field->is_nullable = false; + group_field->field_id = group_schema.__isset.field_id ? group_schema.field_id : -1; } else { RETURN_IF_ERROR(parse_struct_field(t_schemas, curr_pos, group_field)); } @@ -533,6 +547,7 @@ Status FieldDescriptor::parse_list_field(const std::vectortype.type = TYPE_ARRAY; list_field->type.add_sub_type(list_field->children[0].type); list_field->is_nullable = is_optional; + list_field->field_id = first_level.__isset.field_id ? first_level.field_id : -1; return Status::OK(); } @@ -597,6 +612,7 @@ Status FieldDescriptor::parse_map_field(const std::vectortype.add_sub_type(map_kv_field->type.children[0]); map_field->type.add_sub_type(map_kv_field->type.children[1]); map_field->is_nullable = is_optional; + map_field->field_id = map_schema.__isset.field_id ? map_schema.field_id : -1; return Status::OK(); } @@ -619,6 +635,7 @@ Status FieldDescriptor::parse_struct_field(const std::vectorname = to_lower(struct_schema.name); struct_field->is_nullable = is_optional; struct_field->type.type = TYPE_STRUCT; + struct_field->field_id = struct_schema.__isset.field_id ? struct_schema.field_id : -1; for (int i = 0; i < num_children; ++i) { struct_field->type.add_sub_type(struct_field->children[i].type, struct_field->children[i].name); diff --git a/be/src/vec/exec/format/parquet/schema_desc.h b/be/src/vec/exec/format/parquet/schema_desc.h index ca726ef1b57590..2593da837c3da6 100644 --- a/be/src/vec/exec/format/parquet/schema_desc.h +++ b/be/src/vec/exec/format/parquet/schema_desc.h @@ -28,6 +28,7 @@ #include "common/status.h" #include "runtime/types.h" +#include "util/slice.h" namespace doris::vectorized { @@ -56,6 +57,8 @@ struct FieldSchema { ~FieldSchema() = default; FieldSchema(const FieldSchema& fieldSchema) = default; std::string debug_string() const; + + int32_t field_id; }; class FieldDescriptor { @@ -68,6 +71,7 @@ class FieldDescriptor { std::unordered_map _name_to_field; // Used in from_thrift, marking the next schema position that should be parsed size_t _next_schema_pos; + std::unordered_map _field_id_name_mapping; void parse_physical_field(const tparquet::SchemaElement& physical_schema, bool is_nullable, FieldSchema* physical_field); @@ -128,6 +132,10 @@ class FieldDescriptor { std::string debug_string() const; int32_t size() const { return _fields.size(); } + + bool has_parquet_field_id() const { return _field_id_name_mapping.size() > 0; } + + const doris::Slice get_column_name_from_field_id(int32_t id) const; }; } // namespace doris::vectorized diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_reader.cpp index 6c4e4983c70a43..44522454846978 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_reader.cpp @@ -253,10 +253,8 @@ Status ParquetReader::_open_file() { return Status::OK(); } -// Get iceberg col id to col name map stored in parquet metadata key values. -// This is for iceberg schema evolution. -std::vector ParquetReader::get_metadata_key_values() { - return _t_metadata->key_value_metadata; +const FieldDescriptor ParquetReader::get_file_metadata_schema() { + return _file_metadata->schema(); } Status ParquetReader::open() { diff --git a/be/src/vec/exec/format/parquet/vparquet_reader.h b/be/src/vec/exec/format/parquet/vparquet_reader.h index aceb621b825508..00db2652382743 100644 --- a/be/src/vec/exec/format/parquet/vparquet_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_reader.h @@ -148,7 +148,7 @@ class ParquetReader : public GenericReader { partition_columns, const std::unordered_map& missing_columns) override; - std::vector get_metadata_key_values(); + const FieldDescriptor get_file_metadata_schema(); void set_table_to_file_col_map(std::unordered_map& map) { _table_col_to_file_col = map; } diff --git a/be/src/vec/exec/format/table/hudi_jni_reader.cpp b/be/src/vec/exec/format/table/hudi_jni_reader.cpp index 33ba92b540a497..cb109bf05a2393 100644 --- a/be/src/vec/exec/format/table/hudi_jni_reader.cpp +++ b/be/src/vec/exec/format/table/hudi_jni_reader.cpp @@ -18,7 +18,6 @@ #include "hudi_jni_reader.h" #include -#include #include "runtime/descriptors.h" #include "runtime/runtime_state.h" @@ -65,7 +64,7 @@ HudiJniReader::HudiJniReader(const TFileScanRangeParams& scan_params, {"input_format", _hudi_params.input_format}}; // Use compatible hadoop client to read data - for (auto& kv : _scan_params.properties) { + for (const auto& kv : _scan_params.properties) { if (kv.first.starts_with(HOODIE_CONF_PREFIX)) { params[kv.first] = kv.second; } else { @@ -73,8 +72,15 @@ HudiJniReader::HudiJniReader(const TFileScanRangeParams& scan_params, } } - _jni_connector = std::make_unique("org/apache/doris/hudi/HudiJniScanner", params, - required_fields); + if (_hudi_params.hudi_jni_scanner == "hadoop") { + _jni_connector = std::make_unique( + "org/apache/doris/hudi/HadoopHudiJniScanner", params, required_fields); + } else if (_hudi_params.hudi_jni_scanner == "spark") { + _jni_connector = std::make_unique("org/apache/doris/hudi/HudiJniScanner", + params, required_fields); + } else { + DCHECK(false) << "Unsupported hudi jni scanner: " << _hudi_params.hudi_jni_scanner; + } } Status HudiJniReader::get_next_block(Block* block, size_t* read_rows, bool* eof) { diff --git a/be/src/vec/exec/format/table/hudi_jni_reader.h b/be/src/vec/exec/format/table/hudi_jni_reader.h index e9bb55a69a77e7..bfa0291a61035c 100644 --- a/be/src/vec/exec/format/table/hudi_jni_reader.h +++ b/be/src/vec/exec/format/table/hudi_jni_reader.h @@ -17,9 +17,7 @@ #pragma once -#include - -#include +#include #include #include #include diff --git a/be/src/vec/exec/format/table/iceberg_reader.cpp b/be/src/vec/exec/format/table/iceberg_reader.cpp index 295a3a405441b6..8f130ca6002d5d 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.cpp +++ b/be/src/vec/exec/format/table/iceberg_reader.cpp @@ -53,6 +53,7 @@ #include "vec/exec/format/format_common.h" #include "vec/exec/format/generic_reader.h" #include "vec/exec/format/orc/vorc_reader.h" +#include "vec/exec/format/parquet/schema_desc.h" #include "vec/exec/format/table/table_format_reader.h" namespace cctz { @@ -546,8 +547,8 @@ Status IcebergParquetReader::init_reader( _col_id_name_map = col_id_name_map; _file_col_names = file_col_names; _colname_to_value_range = colname_to_value_range; - auto parquet_meta_kv = parquet_reader->get_metadata_key_values(); - RETURN_IF_ERROR(_gen_col_name_maps(parquet_meta_kv)); + FieldDescriptor field_desc = parquet_reader->get_file_metadata_schema(); + RETURN_IF_ERROR(_gen_col_name_maps(field_desc)); _gen_file_col_names(); _gen_new_colname_to_value_range(); parquet_reader->set_table_to_file_col_map(_table_col_to_file_col); @@ -672,39 +673,20 @@ Status IcebergOrcReader::_read_position_delete_file(const TFileRangeDesc* delete * 1. col1_new -> col1 * 2. col1 -> col1_new */ -Status IcebergParquetReader::_gen_col_name_maps(std::vector parquet_meta_kv) { - for (int i = 0; i < parquet_meta_kv.size(); ++i) { - tparquet::KeyValue kv = parquet_meta_kv[i]; - if (kv.key == "iceberg.schema") { - _has_iceberg_schema = true; - std::string schema = kv.value; - rapidjson::Document json; - json.Parse(schema.c_str()); - - if (json.HasMember("fields")) { - rapidjson::Value& fields = json["fields"]; - if (fields.IsArray()) { - for (int j = 0; j < fields.Size(); j++) { - rapidjson::Value& e = fields[j]; - rapidjson::Value& id = e["id"]; - rapidjson::Value& name = e["name"]; - std::string name_string = name.GetString(); - transform(name_string.begin(), name_string.end(), name_string.begin(), - ::tolower); - auto iter = _col_id_name_map.find(id.GetInt()); - if (iter != _col_id_name_map.end()) { - _table_col_to_file_col.emplace(iter->second, name_string); - _file_col_to_table_col.emplace(name_string, iter->second); - if (name_string != iter->second) { - _has_schema_change = true; - } - } else { - _has_schema_change = true; - } - } +Status IcebergParquetReader::_gen_col_name_maps(const FieldDescriptor& field_desc) { + if (field_desc.has_parquet_field_id()) { + for (const auto& pair : _col_id_name_map) { + auto name_slice = field_desc.get_column_name_from_field_id(pair.first); + if (name_slice.get_size() == 0) { + _has_schema_change = true; + } else { + auto name_string = name_slice.to_string(); + _table_col_to_file_col.emplace(pair.second, name_string); + _file_col_to_table_col.emplace(name_string, pair.second); + if (name_string != pair.second) { + _has_schema_change = true; } } - break; } } return Status::OK(); diff --git a/be/src/vec/exec/format/table/iceberg_reader.h b/be/src/vec/exec/format/table/iceberg_reader.h index 04f64aad518a9f..2e240f465b6a2c 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.h +++ b/be/src/vec/exec/format/table/iceberg_reader.h @@ -218,7 +218,7 @@ class IcebergParquetReader final : public IcebergTableReader { parquet_reader->set_delete_rows(&_iceberg_delete_rows); } - Status _gen_col_name_maps(std::vector parquet_meta_kv); + Status _gen_col_name_maps(const FieldDescriptor& field_desc); protected: std::unique_ptr _create_equality_reader( diff --git a/be/src/vec/exec/format/table/max_compute_jni_reader.cpp b/be/src/vec/exec/format/table/max_compute_jni_reader.cpp index d1a71fd1a2f9d9..665e19b6bcebd9 100644 --- a/be/src/vec/exec/format/table/max_compute_jni_reader.cpp +++ b/be/src/vec/exec/format/table/max_compute_jni_reader.cpp @@ -77,7 +77,11 @@ MaxComputeJniReader::MaxComputeJniReader(const MaxComputeTableDescriptor* mc_des {"start_offset", std::to_string(_range.start_offset)}, {"split_size", std::to_string(_range.size)}, {"required_fields", required_fields.str()}, - {"columns_types", columns_types.str()}}; + {"columns_types", columns_types.str()}, + + {"connect_timeout", std::to_string(_max_compute_params.connect_timeout)}, + {"read_timeout", std::to_string(_max_compute_params.read_timeout)}, + {"retry_count", std::to_string(_max_compute_params.retry_times)}}; _jni_connector = std::make_unique( "org/apache/doris/maxcompute/MaxComputeJniScanner", params, column_names); } diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index 451b3561dad46e..a3cafe86daacd1 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -65,7 +65,10 @@ ScannerContext::ScannerContext( _output_row_descriptor->tuple_descriptors().size() == 1); _query_id = _state->get_query_ctx()->query_id(); ctx_id = UniqueId::gen_uid().to_string(); - _scanners.enqueue_bulk(scanners.begin(), scanners.size()); + if (!_scanners.enqueue_bulk(scanners.begin(), scanners.size())) [[unlikely]] { + throw Exception(ErrorCode::INTERNAL_ERROR, + "Exception occurs during scanners initialization."); + }; if (limit < 0) { limit = -1; } diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 385b581d2a5725..3750d8b40b40e2 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -268,7 +268,7 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, } size_t raw_bytes_threshold = config::doris_scanner_row_bytes; - size_t raw_bytes_read = 0; bool first_read = true; + size_t raw_bytes_read = 0; bool first_read = true; int64_t limit = scanner->limit(); while (!eos && raw_bytes_read < raw_bytes_threshold) { if (UNLIKELY(ctx->done())) { eos = true; @@ -322,6 +322,17 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, ctx->inc_block_usage(free_block->allocated_bytes()); scan_task->cached_blocks.emplace_back(std::move(free_block), free_block_bytes); } + if (limit > 0 && limit < ctx->batch_size()) { + // If this scanner has limit, and less than batch size, + // return immediately and no need to wait raw_bytes_threshold. + // This can save time that each scanner may only return a small number of rows, + // but rows are enough from all scanners. + // If not break, the query like "select * from tbl where id=1 limit 10" + // may scan a lot data when the "id=1"'s filter ratio is high. + // If limit is larger than batch size, this rule is skipped, + // to avoid user specify a large limit and causing too much small blocks. + break; + } } // end for while if (UNLIKELY(!status.ok())) { diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 997eef02090912..296e59f8df16e7 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -23,18 +23,15 @@ #include #include -#include #include #include #include -#include #include #include #include "common/compiler_util.h" // IWYU pragma: keep #include "common/config.h" #include "common/logging.h" -#include "common/object_pool.h" #include "io/cache/block_file_cache_profile.h" #include "runtime/descriptors.h" #include "runtime/runtime_state.h" @@ -47,7 +44,6 @@ #include "vec/common/string_ref.h" #include "vec/core/column_with_type_and_name.h" #include "vec/core/columns_with_type_and_name.h" -#include "vec/core/field.h" #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_factory.hpp" #include "vec/data_types/data_type_nullable.h" @@ -720,17 +716,16 @@ Status VFileScanner::_get_next_reader() { // create reader for specific format Status init_status; - TFileFormatType::type format_type = _params->format_type; + // for compatibility, if format_type is not set in range, use the format type of params + TFileFormatType::type format_type = + range.__isset.format_type ? range.format_type : _params->format_type; // JNI reader can only push down column value range bool push_down_predicates = !_is_load && _params->format_type != TFileFormatType::FORMAT_JNI; + // for compatibility, this logic is deprecated in 3.1 if (format_type == TFileFormatType::FORMAT_JNI && range.__isset.table_format_params) { - if (range.table_format_params.table_format_type == "hudi" && - range.table_format_params.hudi_params.delta_logs.empty()) { - // fall back to native reader if there is no log file - format_type = TFileFormatType::FORMAT_PARQUET; - } else if (range.table_format_params.table_format_type == "paimon" && - !range.table_format_params.paimon_params.__isset.paimon_split) { + if (range.table_format_params.table_format_type == "paimon" && + !range.table_format_params.paimon_params.__isset.paimon_split) { // use native reader auto format = range.table_format_params.paimon_params.file_format; if (format == "orc") { @@ -931,8 +926,8 @@ Status VFileScanner::_get_next_reader() { _cur_reader = NewJsonReader::create_unique(_state, _profile, &_counter, *_params, range, _file_slot_descs, &_scanner_eof, _io_ctx.get()); - init_status = - ((NewJsonReader*)(_cur_reader.get()))->init_reader(_col_default_value_ctx); + init_status = ((NewJsonReader*)(_cur_reader.get())) + ->init_reader(_col_default_value_ctx, _is_load); break; } case TFileFormatType::FORMAT_AVRO: { diff --git a/be/src/vec/exec/scan/vscanner.h b/be/src/vec/exec/scan/vscanner.h index 6c4f3294ce1bcc..bb68055e1f07a3 100644 --- a/be/src/vec/exec/scan/vscanner.h +++ b/be/src/vec/exec/scan/vscanner.h @@ -156,6 +156,8 @@ class VScanner { _query_statistics = query_statistics; } + int64_t limit() const { return _limit; } + protected: void _discard_conjuncts() { for (auto& conjunct : _conjuncts) { diff --git a/be/src/vec/functions/function_nullables.cpp b/be/src/vec/functions/function_nullables.cpp index 0fdcfbc2175271..532dc7097c1cc0 100644 --- a/be/src/vec/functions/function_nullables.cpp +++ b/be/src/vec/functions/function_nullables.cpp @@ -96,7 +96,7 @@ class FunctionNonNullable : public IFunction { if (col_null->has_null()) [[unlikely]] { return Status::InvalidArgument( "There's NULL value in column {} which is illegal for non_nullable", - data.column->get_name()); + data.name); } const ColumnPtr& nest_col = col_null->get_nested_column_ptr(); block.replace_by_position(result, nest_col->clone_resized(nest_col->size())); diff --git a/be/src/vec/functions/function_string.cpp b/be/src/vec/functions/function_string.cpp index 1a62c9daaf66f7..1876ed499f4d9e 100644 --- a/be/src/vec/functions/function_string.cpp +++ b/be/src/vec/functions/function_string.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include @@ -508,6 +509,15 @@ struct NameLTrim { struct NameRTrim { static constexpr auto name = "rtrim"; }; +struct NameTrimIn { + static constexpr auto name = "trim_in"; +}; +struct NameLTrimIn { + static constexpr auto name = "ltrim_in"; +}; +struct NameRTrimIn { + static constexpr auto name = "rtrim_in"; +}; template struct TrimUtil { static Status vector(const ColumnString::Chars& str_data, @@ -535,6 +545,135 @@ struct TrimUtil { return Status::OK(); } }; +template +struct TrimInUtil { + static Status vector(const ColumnString::Chars& str_data, + const ColumnString::Offsets& str_offsets, const StringRef& remove_str, + ColumnString::Chars& res_data, ColumnString::Offsets& res_offsets) { + const size_t offset_size = str_offsets.size(); + res_offsets.resize(offset_size); + res_data.reserve(str_data.size()); + bool all_ascii = simd::VStringFunctions::is_ascii(remove_str) && + simd::VStringFunctions::is_ascii(StringRef( + reinterpret_cast(str_data.data()), str_data.size())); + + if (all_ascii) { + return impl_vectors_ascii(str_data, str_offsets, remove_str, res_data, res_offsets); + } else { + return impl_vectors_utf8(str_data, str_offsets, remove_str, res_data, res_offsets); + } + } + +private: + static Status impl_vectors_ascii(const ColumnString::Chars& str_data, + const ColumnString::Offsets& str_offsets, + const StringRef& remove_str, ColumnString::Chars& res_data, + ColumnString::Offsets& res_offsets) { + const size_t offset_size = str_offsets.size(); + std::bitset<128> char_lookup; + const char* remove_begin = remove_str.data; + const char* remove_end = remove_str.data + remove_str.size; + + while (remove_begin < remove_end) { + char_lookup.set(static_cast(*remove_begin)); + remove_begin += 1; + } + + for (size_t i = 0; i < offset_size; ++i) { + const char* str_begin = + reinterpret_cast(str_data.data() + str_offsets[i - 1]); + const char* str_end = reinterpret_cast(str_data.data() + str_offsets[i]); + const char* left_trim_pos = str_begin; + const char* right_trim_pos = str_end; + + if constexpr (is_ltrim) { + while (left_trim_pos < str_end) { + if (!char_lookup.test(static_cast(*left_trim_pos))) { + break; + } + ++left_trim_pos; + } + } + + if constexpr (is_rtrim) { + while (right_trim_pos > left_trim_pos) { + --right_trim_pos; + if (!char_lookup.test(static_cast(*right_trim_pos))) { + ++right_trim_pos; + break; + } + } + } + + res_data.insert_assume_reserved(left_trim_pos, right_trim_pos); + res_offsets[i] = res_data.size(); + } + + return Status::OK(); + } + + static Status impl_vectors_utf8(const ColumnString::Chars& str_data, + const ColumnString::Offsets& str_offsets, + const StringRef& remove_str, ColumnString::Chars& res_data, + ColumnString::Offsets& res_offsets) { + const size_t offset_size = str_offsets.size(); + res_offsets.resize(offset_size); + res_data.reserve(str_data.size()); + + std::unordered_set char_lookup; + const char* remove_begin = remove_str.data; + const char* remove_end = remove_str.data + remove_str.size; + + while (remove_begin < remove_end) { + size_t byte_len, char_len; + std::tie(byte_len, char_len) = simd::VStringFunctions::iterate_utf8_with_limit_length( + remove_begin, remove_end, 1); + char_lookup.insert(std::string_view(remove_begin, byte_len)); + remove_begin += byte_len; + } + + for (size_t i = 0; i < offset_size; ++i) { + const char* str_begin = + reinterpret_cast(str_data.data() + str_offsets[i - 1]); + const char* str_end = reinterpret_cast(str_data.data() + str_offsets[i]); + const char* left_trim_pos = str_begin; + const char* right_trim_pos = str_end; + + if constexpr (is_ltrim) { + while (left_trim_pos < str_end) { + size_t byte_len, char_len; + std::tie(byte_len, char_len) = + simd::VStringFunctions::iterate_utf8_with_limit_length(left_trim_pos, + str_end, 1); + if (char_lookup.find(std::string_view(left_trim_pos, byte_len)) == + char_lookup.end()) { + break; + } + left_trim_pos += byte_len; + } + } + + if constexpr (is_rtrim) { + while (right_trim_pos > left_trim_pos) { + const char* prev_char_pos = right_trim_pos; + do { + --prev_char_pos; + } while ((*prev_char_pos & 0xC0) == 0x80); + size_t byte_len = right_trim_pos - prev_char_pos; + if (char_lookup.find(std::string_view(prev_char_pos, byte_len)) == + char_lookup.end()) { + break; + } + right_trim_pos = prev_char_pos; + } + } + + res_data.insert_assume_reserved(left_trim_pos, right_trim_pos); + res_offsets[i] = res_data.size(); + } + return Status::OK(); + } +}; // This is an implementation of a parameter for the Trim function. template struct Trim1Impl { @@ -583,14 +722,23 @@ struct Trim2Impl { const auto* remove_str_raw = col_right->get_chars().data(); const ColumnString::Offset remove_str_size = col_right->get_offsets()[0]; const StringRef remove_str(remove_str_raw, remove_str_size); + if (remove_str.size == 1) { RETURN_IF_ERROR((TrimUtil::vector( col->get_chars(), col->get_offsets(), remove_str, col_res->get_chars(), col_res->get_offsets()))); } else { - RETURN_IF_ERROR((TrimUtil::vector( - col->get_chars(), col->get_offsets(), remove_str, col_res->get_chars(), - col_res->get_offsets()))); + if constexpr (std::is_same::value || + std::is_same::value || + std::is_same::value) { + RETURN_IF_ERROR((TrimInUtil::vector( + col->get_chars(), col->get_offsets(), remove_str, + col_res->get_chars(), col_res->get_offsets()))); + } else { + RETURN_IF_ERROR((TrimUtil::vector( + col->get_chars(), col->get_offsets(), remove_str, + col_res->get_chars(), col_res->get_offsets()))); + } } block.replace_by_position(result, std::move(col_res)); } else { @@ -746,7 +894,7 @@ struct StringSpace { if (data[i] > 0) { buffer.resize(data[i]); for (size_t j = 0; j < data[i]; ++j) { - buffer[i] = ' '; + buffer[j] = ' '; } StringOP::push_value_string(std::string_view(buffer.data(), buffer.size()), i, res_data, res_offsets); @@ -1023,6 +1171,12 @@ void register_function_string(SimpleFunctionFactory& factory) { factory.register_function>>(); factory.register_function>>(); factory.register_function>>(); + factory.register_function>>(); + factory.register_function>>(); + factory.register_function>>(); + factory.register_function>>(); + factory.register_function>>(); + factory.register_function>>(); factory.register_function(); factory.register_function>(); factory.register_function>(); diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp index 2a38b179b95531..86002233b0c792 100644 --- a/be/src/vec/sink/load_stream_stub.cpp +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -219,11 +219,7 @@ Status LoadStreamStub::append_data(int64_t partition_id, int64_t index_id, int64 add_failed_tablet(tablet_id, _status); return _status; } - DBUG_EXECUTE_IF("LoadStreamStub.only_send_segment_0", { - if (segment_id != 0) { - return Status::OK(); - } - }); + DBUG_EXECUTE_IF("LoadStreamStub.skip_send_segment", { return Status::OK(); }); PStreamHeader header; header.set_src_id(_src_id); *header.mutable_load_id() = _load_id; @@ -246,11 +242,7 @@ Status LoadStreamStub::add_segment(int64_t partition_id, int64_t index_id, int64 add_failed_tablet(tablet_id, _status); return _status; } - DBUG_EXECUTE_IF("LoadStreamStub.only_send_segment_0", { - if (segment_id != 0) { - return Status::OK(); - } - }); + DBUG_EXECUTE_IF("LoadStreamStub.skip_send_segment", { return Status::OK(); }); PStreamHeader header; header.set_src_id(_src_id); *header.mutable_load_id() = _load_id; @@ -340,6 +332,10 @@ Status LoadStreamStub::wait_for_schema(int64_t partition_id, int64_t index_id, i Status LoadStreamStub::close_wait(RuntimeState* state, int64_t timeout_ms) { DBUG_EXECUTE_IF("LoadStreamStub::close_wait.long_wait", DBUG_BLOCK); + if (!_is_open.load()) { + // we don't need to close wait on non-open streams + return Status::OK(); + } if (!_is_closing.load()) { return _status; } diff --git a/be/src/vec/sink/writer/async_result_writer.h b/be/src/vec/sink/writer/async_result_writer.h index 36bca48358a8ab..bc99d8982f547e 100644 --- a/be/src/vec/sink/writer/async_result_writer.h +++ b/be/src/vec/sink/writer/async_result_writer.h @@ -19,7 +19,7 @@ #include #include -#include +#include // IWYU pragma: keep #include "runtime/result_writer.h" #include "vec/exprs/vexpr_fwd.h" @@ -49,7 +49,7 @@ class Block; * pipeline execution engine performance. * * The Sub class of AsyncResultWriter need to impl two virtual function - * * Status open() the first time IO work like: create file/ connect networking + * * Status open() the first time IO work like: create file/ connect network * * Status write() do the real IO work for block */ class AsyncResultWriter : public ResultWriter { @@ -64,7 +64,7 @@ class AsyncResultWriter : public ResultWriter { virtual Status open(RuntimeState* state, RuntimeProfile* profile) = 0; - // sink the block date to date queue, it is async + // sink the block data to data queue, it is async Status sink(Block* block, bool eos); // Add the IO thread task process block() to thread pool to dispose the IO diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index 17f83911a8901b..5c526b9e36bff1 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -269,14 +269,20 @@ Status VTabletWriterV2::open(RuntimeState* state, RuntimeProfile* profile) { } Status VTabletWriterV2::_open_streams() { - bool fault_injection_skip_be = true; + int fault_injection_skip_be = 0; bool any_backend = false; bool any_success = false; for (auto& [dst_id, _] : _tablets_for_node) { auto streams = _load_stream_map->get_or_create(dst_id); DBUG_EXECUTE_IF("VTabletWriterV2._open_streams.skip_one_backend", { - if (fault_injection_skip_be) { - fault_injection_skip_be = false; + if (fault_injection_skip_be < 1) { + fault_injection_skip_be++; + continue; + } + }); + DBUG_EXECUTE_IF("VTabletWriterV2._open_streams.skip_two_backends", { + if (fault_injection_skip_be < 2) { + fault_injection_skip_be++; continue; } }); diff --git a/be/test/http/http_client_test.cpp b/be/test/http/http_client_test.cpp index d42e0a6775ed4b..84e4d259ff5ccd 100644 --- a/be/test/http/http_client_test.cpp +++ b/be/test/http/http_client_test.cpp @@ -25,6 +25,7 @@ #include #include +#include #include "gtest/gtest_pred_impl.h" #include "http/ev_http_server.h" @@ -102,14 +103,32 @@ class HttpDownloadFileHandler : public HttpHandler { } }; +class HttpBatchDownloadFileHandler : public HttpHandler { +public: + void handle(HttpRequest* req) override { + if (req->param("check") == "true") { + HttpChannel::send_reply(req, "OK"); + } else if (req->param("list") == "true") { + do_dir_response(req->param("dir"), req, true); + } else { + std::vector acquire_files = + strings::Split(req->get_request_body(), "\n", strings::SkipWhitespace()); + HttpChannel::send_files(req, req->param("dir"), acquire_files); + } + } +}; + static EvHttpServer* s_server = nullptr; static int real_port = 0; static std::string hostname = ""; +static std::string address = ""; +constexpr std::string_view TMP_DIR = "./http_test_tmp"; static HttpClientTestSimpleGetHandler s_simple_get_handler; static HttpClientTestSimplePostHandler s_simple_post_handler; static HttpNotFoundHandler s_not_found_handler; static HttpDownloadFileHandler s_download_file_handler; +static HttpBatchDownloadFileHandler s_batch_download_file_handler; class HttpClientTest : public testing::Test { public: @@ -123,10 +142,17 @@ class HttpClientTest : public testing::Test { s_server->register_handler(POST, "/simple_post", &s_simple_post_handler); s_server->register_handler(GET, "/not_found", &s_not_found_handler); s_server->register_handler(HEAD, "/download_file", &s_download_file_handler); + s_server->register_handler(HEAD, "/api/_tablet/_batch_download", + &s_batch_download_file_handler); + s_server->register_handler(GET, "/api/_tablet/_batch_download", + &s_batch_download_file_handler); + s_server->register_handler(POST, "/api/_tablet/_batch_download", + &s_batch_download_file_handler); static_cast(s_server->start()); real_port = s_server->get_real_port(); EXPECT_NE(0, real_port); - hostname = "http://127.0.0.1:" + std::to_string(real_port); + address = "127.0.0.1:" + std::to_string(real_port); + hostname = "http://" + address; } static void TearDownTestCase() { delete s_server; } @@ -413,7 +439,7 @@ TEST_F(HttpClientTest, enable_http_auth) { EXPECT_TRUE(!st.ok()); std::cout << "response = " << response << "\n"; std::cout << "st.msg() = " << st.msg() << "\n"; - EXPECT_TRUE(st.msg().find("Operation timed out after") != std::string::npos); + EXPECT_TRUE(st.msg().find("403") != std::string::npos); } { @@ -474,7 +500,7 @@ TEST_F(HttpClientTest, enable_http_auth) { EXPECT_TRUE(!st.ok()); std::cout << "response = " << response << "\n"; std::cout << "st.msg() = " << st.msg() << "\n"; - EXPECT_TRUE(st.msg().find("Operation timed out after") != std::string::npos); + EXPECT_TRUE(st.msg().find("403") != std::string::npos); } // valid token @@ -521,7 +547,7 @@ TEST_F(HttpClientTest, enable_http_auth) { EXPECT_TRUE(!st.ok()); std::cout << "response = " << response << "\n"; std::cout << "st.msg() = " << st.msg() << "\n"; - EXPECT_TRUE(st.msg().find("Operation timed out after") != std::string::npos); + EXPECT_TRUE(st.msg().find("403") != std::string::npos); } std::vector check_get_list = {"/api/clear_cache/aa", @@ -566,9 +592,79 @@ TEST_F(HttpClientTest, enable_http_auth) { EXPECT_TRUE(!st.ok()); std::cout << "response = " << response << "\n"; std::cout << "st.msg() = " << st.msg() << "\n"; - EXPECT_TRUE(st.msg().find("Operation timed out after") != std::string::npos); + EXPECT_TRUE(st.msg().find("403") != std::string::npos); + } + } +} + +TEST_F(HttpClientTest, batch_download) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(TMP_DIR).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(TMP_DIR).ok()); + + std::string root_dir(TMP_DIR); + std::string remote_related_dir = root_dir + "/source"; + std::string local_dir = root_dir + "/target"; + EXPECT_TRUE(io::global_local_filesystem()->create_directory(remote_related_dir).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(local_dir).ok()); + + std::string remote_dir; + EXPECT_TRUE(io::global_local_filesystem()->canonicalize(remote_related_dir, &remote_dir).ok()); + + // 0. create dir source and prepare a large file exceeds 1MB + { + std::string large_file = remote_dir + "/a_large_file"; + int fd = open(large_file.c_str(), O_CREAT | O_RDWR | O_TRUNC, 0644); + ASSERT_TRUE(fd >= 0); + std::string buf = "0123456789"; + for (int i = 0; i < 10; i++) { + buf += buf; + } + for (int i = 0; i < 1024; i++) { + ASSERT_TRUE(write(fd, buf.c_str(), buf.size()) > 0); } + close(fd); + + // create some small files. + for (int i = 0; i < 32; i++) { + std::string small_file = remote_dir + "/small_file_" + std::to_string(i); + fd = open(small_file.c_str(), O_CREAT | O_RDWR | O_TRUNC, 0644); + ASSERT_TRUE(fd >= 0); + ASSERT_TRUE(write(fd, buf.c_str(), buf.size()) > 0); + close(fd); + } + + // create a empty file + std::string empty_file = remote_dir + "/empty_file"; + fd = open(empty_file.c_str(), O_CREAT | O_RDWR | O_TRUNC, 0644); + ASSERT_TRUE(fd >= 0); + close(fd); + + empty_file = remote_dir + "/zzzz"; + fd = open(empty_file.c_str(), O_CREAT | O_RDWR | O_TRUNC, 0644); + ASSERT_TRUE(fd >= 0); + close(fd); + } + + // 1. check remote support batch download + Status st = is_support_batch_download(address); + EXPECT_TRUE(st.ok()); + + // 2. list remote files + std::vector> file_info_list; + st = list_remote_files_v2(address, "token", remote_dir, &file_info_list); + EXPECT_TRUE(st.ok()); + + // 3. download files + if (file_info_list.size() > 64) { + file_info_list.resize(64); } + + // sort file info list by file name + std::sort(file_info_list.begin(), file_info_list.end(), + [](const auto& a, const auto& b) { return a.first < b.first; }); + + st = download_files_v2(address, "token", remote_dir, local_dir, file_info_list); + EXPECT_TRUE(st.ok()); } } // namespace doris diff --git a/be/test/io/fs/local_file_system_test.cpp b/be/test/io/fs/local_file_system_test.cpp index 0fd18445beac4c..c930ba72eabf86 100644 --- a/be/test/io/fs/local_file_system_test.cpp +++ b/be/test/io/fs/local_file_system_test.cpp @@ -417,4 +417,54 @@ TEST_F(LocalFileSystemTest, TestGlob) { EXPECT_TRUE(io::global_local_filesystem()->delete_directory(path).ok()); } +TEST_F(LocalFileSystemTest, TestConvertToAbsPath) { + io::Path abs_path; + Status st; + + // suppurt path: + st = doris::io::LocalFileSystem::convert_to_abs_path("/abc/def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/abc/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:/def/hij", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/def/hij", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://host:80/hij/abc", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/hij/abc", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://host/abc/def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/abc/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:///def", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/def", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file:///", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("file://auth/", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("/", abs_path); + + st = doris::io::LocalFileSystem::convert_to_abs_path("abc", abs_path); + ASSERT_TRUE(st.ok()); + ASSERT_EQ("abc", abs_path); + + // not support path: + st = doris::io::LocalFileSystem::convert_to_abs_path("file://auth", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("fileee:/abc", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("hdfs:///abc", abs_path); + ASSERT_TRUE(!st.ok()); + + st = doris::io::LocalFileSystem::convert_to_abs_path("hdfs:/abc", abs_path); + ASSERT_TRUE(!st.ok()); +} } // namespace doris diff --git a/be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp new file mode 100644 index 00000000000000..96624260521ab5 --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index/common/inverted_index_common_test.cpp @@ -0,0 +1,343 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "olap/rowset/segment_v2/inverted_index_common.h" + +#include + +#include "common/status.h" + +namespace doris::segment_v2 { + +class InvertedIndexCommonTest : public testing::Test { +public: + void SetUp() override {} + + void TearDown() override {} + + InvertedIndexCommonTest() = default; + ~InvertedIndexCommonTest() override = default; +}; + +TEST_F(InvertedIndexCommonTest, TestFinallyClose) { + class InvertedIndexBase { + public: + InvertedIndexBase(int32_t& count) : count_(count) {} + + void close() { count_++; } + void clear() { count_++; } + + int32_t& count_; + }; + { + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::make_shared(count); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 1); + } + { + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::shared_ptr(new InvertedIndexBase(count), + [](InvertedIndexBase* p) { + if (p) { + p->clear(); + delete p; + p = nullptr; + } + }); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 2); + } + { + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::make_unique(count); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 1); + } + { + struct Deleter { + void operator()(InvertedIndexBase* p) const { + if (p) { + p->clear(); + delete p; + p = nullptr; + } + } + }; + + int32_t count = 0; + { + ErrorContext error_context; + auto ptr = std::unique_ptr(new InvertedIndexBase(count)); + finally_close(ptr, error_context); + } + EXPECT_EQ(count, 2); + } +} + +TEST_F(InvertedIndexCommonTest, TestTryBlockException) { + class InvertedIndexBase { + public: + void add() { _CLTHROWA(CL_ERR_IO, "test add error"); } + void close() {} + }; + + // return error + { + auto func = []() -> Status { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_IO); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +TEST_F(InvertedIndexCommonTest, TestFinallyBlockException) { + class InvertedIndexBase { + public: + void add() {} + void close() { _CLTHROWA(CL_ERR_Runtime, "test close error"); } + }; + + // return error + { + auto func = []() -> Status { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_FALSE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_FALSE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_Runtime); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +TEST_F(InvertedIndexCommonTest, TestTryAndFinallyBlockException) { + class InvertedIndexBase { + public: + void add() { _CLTHROWA(CL_ERR_IO, "test add error"); } + void close() { _CLTHROWA(CL_ERR_Runtime, "test close error"); } + }; + + // return error + { + auto func = []() -> Status { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto base_ptr = std::make_unique(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + EXPECT_TRUE(error_context.eptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_Runtime); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +TEST_F(InvertedIndexCommonTest, TestRawPointerException) { + class InvertedIndexBase { + public: + void add() { _CLTHROWA(CL_ERR_IO, "test add error"); } + void close() { _CLTHROWA(CL_ERR_Runtime, "test close error"); } + }; + + // return error + { + auto func = []() -> Status { + auto* base_ptr = new InvertedIndexBase(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + if (base_ptr) { + delete base_ptr; + base_ptr = nullptr; + } + EXPECT_TRUE(error_context.eptr); + }) + return Status::OK(); + }; + auto ret = func(); + EXPECT_EQ(ret.code(), ErrorCode::INVERTED_INDEX_CLUCENE_ERROR); + } + + // throw exception + { + auto func = []() { + auto* base_ptr = new InvertedIndexBase(); + ErrorContext error_context; + try { + base_ptr->add(); + } catch (CLuceneError& e) { + error_context.eptr = std::current_exception(); + error_context.err_msg.append("error: "); + error_context.err_msg.append(e.what()); + } + FINALLY_EXCEPTION({ + EXPECT_TRUE(error_context.eptr); + FINALLY_CLOSE(base_ptr); + if (base_ptr) { + delete base_ptr; + base_ptr = nullptr; + } + EXPECT_TRUE(error_context.eptr); + }) + }; + bool is_exception = false; + try { + func(); + } catch (CLuceneError& e) { + EXPECT_EQ(e.number(), CL_ERR_Runtime); + is_exception = true; + } + EXPECT_TRUE(is_exception); + } +} + +} // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp index 0f1b27fd4fa11f..264786570e731a 100644 --- a/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp +++ b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_test.cpp @@ -15,14 +15,9 @@ // specific language governing permissions and limitations // under the License. -#include +#include -#include "olap/cumulative_compaction.h" -#include "olap/rowset/beta_rowset_writer.h" -#include "olap/rowset/rowset_factory.h" -#include "olap/rowset/segment_v2/inverted_index/query/query_factory.h" -#include "olap/rowset/segment_v2/inverted_index_file_reader.h" -#include "olap/storage_engine.h" +#include "util/index_compaction_utils.cpp" namespace doris { @@ -31,14 +26,6 @@ using namespace doris::vectorized; constexpr static uint32_t MAX_PATH_LEN = 1024; constexpr static std::string_view dest_dir = "./ut_dir/inverted_index_test"; constexpr static std::string_view tmp_dir = "./ut_dir/tmp"; -static int64_t inc_id = 1000; - -struct DataRow { - int key; - std::string word; - std::string url; - int num; -}; class IndexCompactionTest : public ::testing::Test { protected: @@ -46,8 +33,8 @@ class IndexCompactionTest : public ::testing::Test { // absolute dir char buffer[MAX_PATH_LEN]; EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr); - _curreent_dir = std::string(buffer); - _absolute_dir = _curreent_dir + std::string(dest_dir); + _current_dir = std::string(buffer); + _absolute_dir = _current_dir + std::string(dest_dir); EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_absolute_dir).ok()); EXPECT_TRUE(io::global_local_filesystem()->create_directory(_absolute_dir).ok()); @@ -57,7 +44,8 @@ class IndexCompactionTest : public ::testing::Test { std::vector paths; paths.emplace_back(std::string(tmp_dir), 1024000000); auto tmp_file_dirs = std::make_unique(paths); - EXPECT_TRUE(tmp_file_dirs->init().ok()); + Status st = tmp_file_dirs->init(); + EXPECT_TRUE(st.ok()) << st.to_json(); ExecEnv::GetInstance()->set_tmp_file_dir(std::move(tmp_file_dirs)); // storage engine @@ -73,24 +61,22 @@ class IndexCompactionTest : public ::testing::Test { schema_pb.set_keys_type(KeysType::DUP_KEYS); schema_pb.set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V2); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10000, "key_index", 0, - "INT", "key"); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10001, "v1_index", 1, - "STRING", "v1"); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10002, "v2_index", 2, - "STRING", "v2", true); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10003, "v3_index", 3, "INT", - "v3"); - - _tablet_schema.reset(new TabletSchema); + IndexCompactionUtils::construct_column(schema_pb.add_column(), schema_pb.add_index(), 10000, + "key_index", 0, "INT", "key"); + IndexCompactionUtils::construct_column(schema_pb.add_column(), schema_pb.add_index(), 10001, + "v1_index", 1, "STRING", "v1"); + IndexCompactionUtils::construct_column(schema_pb.add_column(), schema_pb.add_index(), 10002, + "v2_index", 2, "STRING", "v2", true); + IndexCompactionUtils::construct_column(schema_pb.add_column(), schema_pb.add_index(), 10003, + "v3_index", 3, "INT", "v3"); + _tablet_schema = std::make_shared(); _tablet_schema->init_from_pb(schema_pb); // tablet TabletMetaSharedPtr tablet_meta(new TabletMeta(_tablet_schema)); - _tablet.reset(new Tablet(*_engine_ref, tablet_meta, _data_dir.get())); + _tablet = std::make_shared(*_engine_ref, tablet_meta, _data_dir.get()); EXPECT_TRUE(_tablet->init().ok()); - config::inverted_index_compaction_enable = true; } void TearDown() override { EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); @@ -100,41 +86,6 @@ class IndexCompactionTest : public ::testing::Test { ExecEnv::GetInstance()->set_storage_engine(nullptr); } - void construct_column(ColumnPB* column_pb, TabletIndexPB* tablet_index, int64_t index_id, - const std::string& index_name, int32_t col_unique_id, - const std::string& column_type, const std::string& column_name, - bool parser = false) { - column_pb->set_unique_id(col_unique_id); - column_pb->set_name(column_name); - column_pb->set_type(column_type); - column_pb->set_is_key(false); - column_pb->set_is_nullable(true); - tablet_index->set_index_id(index_id); - tablet_index->set_index_name(index_name); - tablet_index->set_index_type(IndexType::INVERTED); - tablet_index->add_col_unique_id(col_unique_id); - if (parser) { - auto* properties = tablet_index->mutable_properties(); - (*properties)[INVERTED_INDEX_PARSER_KEY] = INVERTED_INDEX_PARSER_UNICODE; - } - } - - RowsetWriterContext rowset_writer_context() { - RowsetWriterContext context; - RowsetId rowset_id; - rowset_id.init(inc_id); - context.rowset_id = rowset_id; - context.rowset_type = BETA_ROWSET; - context.data_dir = _data_dir.get(); - context.rowset_state = VISIBLE; - context.tablet_schema = _tablet_schema; - context.tablet_path = _tablet->tablet_path(); - context.version = Version(inc_id, inc_id); - context.max_rows_per_segment = 200; - inc_id++; - return context; - } - IndexCompactionTest() = default; ~IndexCompactionTest() override = default; @@ -144,300 +95,640 @@ class IndexCompactionTest : public ::testing::Test { std::unique_ptr _data_dir = nullptr; TabletSharedPtr _tablet = nullptr; std::string _absolute_dir; - std::string _curreent_dir; + std::string _current_dir; }; -std::vector read_data(const std::string file_name) { - std::ifstream file(file_name); - EXPECT_TRUE(file.is_open()); - - std::string line; - std::vector data; - - while (std::getline(file, line)) { - std::stringstream ss(line); - std::string item; - DataRow row; - EXPECT_TRUE(std::getline(ss, item, ',')); - row.key = std::stoi(item); - EXPECT_TRUE(std::getline(ss, item, ',')); - row.word = item; - EXPECT_TRUE(std::getline(ss, item, ',')); - row.url = item; - EXPECT_TRUE(std::getline(ss, item, ',')); - row.num = std::stoi(item); - data.emplace_back(std::move(row)); - } +TEST_F(IndexCompactionTest, tes_write_index_normally) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 2); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); + + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + auto custom_check_normal = [](const BaseCompaction& compaction, + const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 0); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_normal; + st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, false, + output_rowset_normal, custom_check_normal); + EXPECT_TRUE(st.ok()) << st.to_string(); + const auto& seg_path_normal = output_rowset_normal->segment_path(0); + EXPECT_TRUE(seg_path_normal.has_value()) << seg_path_normal.error(); + auto inverted_index_file_reader_normal = IndexCompactionUtils::init_index_file_reader( + output_rowset_normal, seg_path_normal.value(), + _tablet_schema->get_inverted_index_storage_format()); + // check index file terms + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + auto dir_normal_compaction = inverted_index_file_reader_normal->_open(10001, ""); + EXPECT_TRUE(dir_normal_compaction.has_value()) << dir_normal_compaction.error(); + std::ostringstream oss; + IndexCompactionUtils::check_terms_stats(dir_idx_compaction->get(), oss); + std::string output = oss.str(); + EXPECT_EQ(output, expected_output); + oss.str(""); + oss.clear(); + IndexCompactionUtils::check_terms_stats(dir_normal_compaction->get(), oss); + output = oss.str(); + EXPECT_EQ(output, expected_output); + + st = IndexCompactionUtils::check_idx_file_correctness(dir_idx_compaction->get(), + dir_normal_compaction->get()); + EXPECT_TRUE(st.ok()) << st.to_string(); - file.close(); - return data; + // check meta and file + std::map query_map = { + {0, {{"99", "66", "56", "87", "85", "96", "20000"}, {21, 25, 22, 18, 14, 18, 0}}}, + {3, {{"99", "66", "56", "87", "85", "96", "10000"}, {12, 20, 25, 23, 16, 24, 0}}}, + {1, {{"good", "maybe", "great", "null"}, {197, 191, 194, 0}}}, + {2, {{"musicstream.com", "http", "https", "null"}, {191, 799, 1201, 0}}}}; + IndexCompactionUtils::check_meta_and_file(output_rowset_index, _tablet_schema, query_map); + IndexCompactionUtils::check_meta_and_file(output_rowset_normal, _tablet_schema, query_map); } -bool query_bkd(const TabletIndex* index, - std::shared_ptr& inverted_index_file_reader, - const std::vector& query_data, const std::vector& query_result) { - const auto& idx_reader = BkdIndexReader::create_shared(index, inverted_index_file_reader); - const auto& index_searcher_builder = std::make_unique(); - auto dir = inverted_index_file_reader->open(index); - EXPECT_TRUE(dir.has_value()); - auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); - EXPECT_TRUE(searcher_result.has_value()); - auto bkd_searcher = std::get_if(&searcher_result.value()); - EXPECT_TRUE(bkd_searcher != nullptr); - idx_reader->_type_info = get_scalar_type_info((FieldType)(*bkd_searcher)->type); - EXPECT_TRUE(idx_reader->_type_info != nullptr); - idx_reader->_value_key_coder = get_key_coder(idx_reader->_type_info->type()); - - for (int i = 0; i < query_data.size(); i++) { - vectorized::Field param_value = Int32(query_data[i]); - std::unique_ptr query_param = nullptr; - EXPECT_TRUE(segment_v2::InvertedIndexQueryParamFactory::create_query_value( - PrimitiveType::TYPE_INT, ¶m_value, query_param) - .ok()); - auto result = std::make_shared(); - EXPECT_TRUE(idx_reader - ->invoke_bkd_query(query_param->get_value(), - InvertedIndexQueryType::EQUAL_QUERY, *bkd_searcher, - result) - .ok()); - EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; - } - return true; +TEST_F(IndexCompactionTest, test_col_unique_ids_empty) { + // clear column unique id in tablet index 10001 and rebuild tablet_schema + TabletSchemaPB schema_pb; + _tablet_schema->to_schema_pb(&schema_pb); + auto* index_pb = schema_pb.mutable_index(1); + index_pb->clear_col_unique_id(); + _tablet_schema->init_from_pb(schema_pb); + + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 3); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); + + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + // index 10001 cannot be found in idx file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(!dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + EXPECT_THAT(dir_idx_compaction.error().to_string(), + testing::HasSubstr("No index with id 10001 found")); } -bool query_string(const TabletIndex* index, - std::shared_ptr& inverted_index_file_reader, - const std::string& column_name, const std::vector& query_data, - const std::vector& query_result) { - const auto& idx_reader = - StringTypeInvertedIndexReader::create_shared(index, inverted_index_file_reader); - const auto& index_searcher_builder = std::make_unique(); - auto dir = inverted_index_file_reader->open(index); - EXPECT_TRUE(dir.has_value()); - auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); - EXPECT_TRUE(searcher_result.has_value()); - auto string_searcher = std::get_if(&searcher_result.value()); - EXPECT_TRUE(string_searcher != nullptr); - std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); - - for (int i = 0; i < query_data.size(); i++) { - TQueryOptions queryOptions; - auto query = QueryFactory::create(InvertedIndexQueryType::EQUAL_QUERY, *string_searcher, - queryOptions, nullptr); - EXPECT_TRUE(query != nullptr); - InvertedIndexQueryInfo query_info; - query_info.field_name = column_name_ws; - query_info.terms.emplace_back(query_data[i]); - query->add(query_info); - auto result = std::make_shared(); - query->search(*result); - EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; - } - return true; +TEST_F(IndexCompactionTest, test_tablet_index_id_not_equal) { + // replace unique id from 2 to 1 in tablet index 10002 and rebuild tablet_schema + TabletSchemaPB schema_pb; + _tablet_schema->to_schema_pb(&schema_pb); + auto* index_pb = schema_pb.mutable_index(2); + index_pb->set_col_unique_id(0, 1); + _tablet_schema->init_from_pb(schema_pb); + + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 3); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10001 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); + + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + // index 10002 cannot be found in idx file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10002, ""); + EXPECT_TRUE(!dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + EXPECT_THAT(dir_idx_compaction.error().to_string(), + testing::HasSubstr("No index with id 10002 found")); } -bool query_fulltext(const TabletIndex* index, - std::shared_ptr& inverted_index_file_reader, - const std::string& column_name, const std::vector& query_data, - const std::vector& query_result) { - const auto& idx_reader = FullTextIndexReader::create_shared(index, inverted_index_file_reader); - const auto& index_searcher_builder = std::make_unique(); - auto dir = inverted_index_file_reader->open(index); - EXPECT_TRUE(dir.has_value()); - auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); - EXPECT_TRUE(searcher_result.has_value()); - auto string_searcher = std::get_if(&searcher_result.value()); - EXPECT_TRUE(string_searcher != nullptr); - std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); - - for (int i = 0; i < query_data.size(); i++) { - TQueryOptions queryOptions; - auto query = QueryFactory::create(InvertedIndexQueryType::MATCH_ANY_QUERY, *string_searcher, - queryOptions, nullptr); - EXPECT_TRUE(query != nullptr); - InvertedIndexQueryInfo query_info; - query_info.field_name = column_name_ws; - query_info.terms.emplace_back(query_data[i]); - query->add(query_info); - auto result = std::make_shared(); - query->search(*result); - EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; - } - return true; +TEST_F(IndexCompactionTest, test_tablet_schema_tablet_index_is_null) { + // set index suffix in tablet index 10001 and rebuild tablet_schema + // simulate the case that index is null, tablet_schema->inverted_index(1) will return nullptr + TabletSchemaPB schema_pb; + _tablet_schema->to_schema_pb(&schema_pb); + auto* index_pb = schema_pb.mutable_index(1); + index_pb->set_index_suffix_name("mock"); + _tablet_schema->init_from_pb(schema_pb); + + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 3); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); + + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + // index 10001 cannot be found in idx file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(!dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + EXPECT_THAT(dir_idx_compaction.error().to_string(), + testing::HasSubstr("No index with id 10001 found")); } -TEST_F(IndexCompactionTest, write_index_test) { +TEST_F(IndexCompactionTest, test_rowset_schema_tablet_index_is_null) { EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); std::string data_file1 = - _curreent_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; std::string data_file2 = - _curreent_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; - - std::vector> data; - data.emplace_back(read_data(data_file1)); - data.emplace_back(read_data(data_file2)); - - std::vector rowsets(data.size()); - for (int i = 0; i < data.size(); i++) { - const auto& res = - RowsetFactory::create_rowset_writer(*_engine_ref, rowset_writer_context(), false); - EXPECT_TRUE(res.has_value()) << res.error(); - const auto& rowset_writer = res.value(); - - Block block = _tablet_schema->create_block(); - auto columns = block.mutate_columns(); - for (const auto& row : data[i]) { - vectorized::Field key = Int32(row.key); - vectorized::Field v1(row.word); - vectorized::Field v2(row.url); - vectorized::Field v3 = Int32(row.num); - columns[0]->insert(key); - columns[1]->insert(v1); - columns[2]->insert(v2); - columns[3]->insert(v3); - } - EXPECT_TRUE(rowset_writer->add_block(&block).ok()); - EXPECT_TRUE(rowset_writer->flush().ok()); - const auto& dst_writer = dynamic_cast(rowset_writer.get()); - - // inverted index file writer - for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { - EXPECT_TRUE(idx_file_writer->_closed); - } - - EXPECT_TRUE(rowset_writer->build(rowsets[i]).ok()); - EXPECT_TRUE(_tablet->add_rowset(rowsets[i]).ok()); - EXPECT_TRUE(rowsets[i]->num_segments() == 5); - - // check rowset meta and file - for (int seg_id = 0; seg_id < rowsets[i]->num_segments(); seg_id++) { - const auto& index_info = rowsets[i]->_rowset_meta->inverted_index_file_info(seg_id); - EXPECT_TRUE(index_info.has_index_size()); - const auto& fs = rowsets[i]->_rowset_meta->fs(); - const auto& file_name = fmt::format("{}/{}_{}.idx", rowsets[i]->tablet_path(), - rowsets[i]->rowset_id().to_string(), seg_id); - int64_t file_size = 0; - EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); - EXPECT_EQ(index_info.index_size(), file_size); - - const auto& seg_path = rowsets[i]->segment_path(seg_id); - EXPECT_TRUE(seg_path.has_value()); - const auto& index_file_path_prefix = - InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); - auto inverted_index_file_reader = std::make_shared( - fs, std::string(index_file_path_prefix), - _tablet_schema->get_inverted_index_storage_format(), index_info); - EXPECT_TRUE(inverted_index_file_reader->init().ok()); - const auto& dirs = inverted_index_file_reader->get_all_directories(); - EXPECT_TRUE(dirs.has_value()); - EXPECT_EQ(dirs.value().size(), 4); - } - } + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + // set index suffix in tablet index 10001 and rebuild tablet_schema + // simulate the case that index is null, tablet_schema->inverted_index(1) will return nullptr + TabletSchemaPB schema_pb; + TabletSchemaSPtr mock_schema = std::make_shared(); + _tablet_schema->to_schema_pb(&schema_pb); + auto* index_pb = schema_pb.mutable_index(1); + index_pb->set_index_suffix_name("mock"); + mock_schema->init_from_pb(schema_pb); + rowsets[0]->_schema = mock_schema; + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); - CumulativeCompaction compaction(*_engine_ref, _tablet); - compaction._input_rowsets = std::move(rowsets); - compaction.build_basic_info(); + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + // index 10001 cannot be found in idx file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + // check index 10001 term stats + std::ostringstream oss; + IndexCompactionUtils::check_terms_stats(dir_idx_compaction.value().get(), oss); + std::string output = oss.str(); + EXPECT_EQ(output, expected_output); +} - std::vector input_rs_readers; - input_rs_readers.reserve(compaction._input_rowsets.size()); - for (auto& rowset : compaction._input_rowsets) { - RowsetReaderSharedPtr rs_reader; - EXPECT_TRUE(rowset->create_reader(&rs_reader).ok()); - input_rs_readers.push_back(std::move(rs_reader)); - } +TEST_F(IndexCompactionTest, test_tablet_index_properties_not_equal) { + // add mock property in tablet index 10001 and rebuild tablet_schema + // simulate the case that index properties not equal among input rowsets + TabletSchemaSPtr mock_schema = std::make_shared(); + TabletSchemaPB schema_pb; + _tablet_schema->to_schema_pb(&schema_pb); + auto* index_pb = schema_pb.mutable_index(1); + (*index_pb->mutable_properties())["mock_key"] = "mock_value"; + mock_schema->init_from_pb(schema_pb); - RowsetWriterContext ctx; - EXPECT_TRUE(compaction.construct_output_rowset_writer(ctx).ok()); - - // col word - EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); - // col url - EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); - - compaction._stats.rowid_conversion = compaction._rowid_conversion.get(); - EXPECT_TRUE(Merger::vertical_merge_rowsets(_tablet, compaction.compaction_type(), - *(compaction._cur_tablet_schema), input_rs_readers, - compaction._output_rs_writer.get(), 100000, 5, - &compaction._stats) - .ok()); - const auto& dst_writer = - dynamic_cast(compaction._output_rs_writer.get()); - for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { - EXPECT_FALSE(idx_file_writer->_closed); - } - auto st = compaction.do_inverted_index_compaction(); + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + // set mock_schema to the first input rowset + rowsets[0]->_schema = mock_schema; + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); EXPECT_TRUE(st.ok()) << st.to_string(); - st = compaction._output_rs_writer->build(compaction._output_rowset); + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + + // check index 10001 term stats + std::ostringstream oss; + IndexCompactionUtils::check_terms_stats(dir_idx_compaction.value().get(), oss); + std::string output = oss.str(); + EXPECT_EQ(output, expected_output); +} + +TEST_F(IndexCompactionTest, test_is_skip_index_compaction_not_empty) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + // set col_unique_id=1(index_id=10001) to skip index compaction + rowsets[0]->set_skip_index_compaction(1); + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); EXPECT_TRUE(st.ok()) << st.to_string(); - for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { - EXPECT_TRUE(idx_file_writer->_closed); - } - EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); - - const auto& output_rowset = compaction._output_rowset; - - // check rowset meta and file - for (int seg_id = 0; seg_id < output_rowset->num_segments(); seg_id++) { - // meta - const auto& index_info = output_rowset->_rowset_meta->inverted_index_file_info(seg_id); - EXPECT_TRUE(index_info.has_index_size()); - const auto& fs = output_rowset->_rowset_meta->fs(); - const auto& file_name = fmt::format("{}/{}_{}.idx", output_rowset->tablet_path(), - output_rowset->rowset_id().to_string(), seg_id); - int64_t file_size = 0; - EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); - EXPECT_EQ(index_info.index_size(), file_size); - - // file - const auto& seg_path = output_rowset->segment_path(seg_id); - EXPECT_TRUE(seg_path.has_value()); - const auto& index_file_path_prefix = - InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); - auto inverted_index_file_reader = std::make_shared( - fs, std::string(index_file_path_prefix), - _tablet_schema->get_inverted_index_storage_format(), index_info); - EXPECT_TRUE(inverted_index_file_reader->init().ok()); - const auto& dirs = inverted_index_file_reader->get_all_directories(); - EXPECT_TRUE(dirs.has_value()); - EXPECT_EQ(dirs.value().size(), 4); - - // read col key - const auto& key = _tablet_schema->column_by_uid(0); - const auto* key_index = _tablet_schema->inverted_index(key); - EXPECT_TRUE(key_index != nullptr); - std::vector query_data {99, 66, 56, 87, 85, 96, 20000}; - std::vector query_result {21, 25, 22, 18, 14, 18, 0}; - EXPECT_TRUE(query_bkd(key_index, inverted_index_file_reader, query_data, query_result)); - - // read col v3 - const auto& v3_column = _tablet_schema->column_by_uid(3); - const auto* v3_index = _tablet_schema->inverted_index(v3_column); - EXPECT_TRUE(v3_index != nullptr); - std::vector query_data3 {99, 66, 56, 87, 85, 96, 10000}; - std::vector query_result3 {12, 20, 25, 23, 16, 24, 0}; - EXPECT_TRUE(query_bkd(v3_index, inverted_index_file_reader, query_data3, query_result3)); - - // read col v1 - const auto& v1_column = _tablet_schema->column_by_uid(1); - const auto* v1_index = _tablet_schema->inverted_index(v1_column); - EXPECT_TRUE(v1_index != nullptr); - std::vector query_data1 {"good", "maybe", "great", "null"}; - std::vector query_result1 {197, 191, 194, 0}; - EXPECT_TRUE(query_string(v1_index, inverted_index_file_reader, "1", query_data1, - query_result1)); - - // read col v2 - const auto& v2_column = _tablet_schema->column_by_uid(2); - const auto* v2_index = _tablet_schema->inverted_index(v2_column); - EXPECT_TRUE(v2_index != nullptr); - std::vector query_data2 {"musicstream.com", "http", "https", "null"}; - std::vector query_result2 {191, 799, 1201, 0}; - EXPECT_TRUE(query_fulltext(v2_index, inverted_index_file_reader, "2", query_data2, - query_result2)); + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + + // check index 10001 term stats + std::ostringstream oss; + IndexCompactionUtils::check_terms_stats(dir_idx_compaction.value().get(), oss); + std::string output = oss.str(); + EXPECT_EQ(output, expected_output); +} + +TEST_F(IndexCompactionTest, test_rowset_fs_nullptr) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 1); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + // set mock_id to resource_id to simulate getting fs nullptr + RowsetMetaSharedPtr mock_rowset_meta = std::make_shared(); + RowsetMetaPB rs_meta_pb; + rowsets[0]->to_rowset_pb(&rs_meta_pb); + rs_meta_pb.set_resource_id("mock_id"); + mock_rowset_meta->init_from_pb(rs_meta_pb); + rowsets[0]->_rowset_meta = mock_rowset_meta; + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(!st.ok()); + EXPECT_THAT(st.to_string(), testing::HasSubstr("[E-206]get fs failed")); +} + +TEST_F(IndexCompactionTest, test_input_row_num_zero) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // only index id 10002 will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 2); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + // set num_rows to 0 to simulate input_row_num = 0 + for (auto rowset : rowsets) { + RowsetMetaSharedPtr mock_rowset_meta = std::make_shared(); + RowsetMetaPB rs_meta_pb; + rowset->to_rowset_pb(&rs_meta_pb); + rs_meta_pb.set_num_rows(0); + mock_rowset_meta->init_from_pb(rs_meta_pb); + rowset->_rowset_meta = mock_rowset_meta; } + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()); + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + // index 10001 cannot be found in idx file + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(!dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + EXPECT_THAT(dir_idx_compaction.error().to_string(), + testing::HasSubstr("No index with id 10001 found")); +} + +TEST_F(IndexCompactionTest, test_cols_to_do_index_compaction_empty) { + // add mock property in tablet index 10001, 10002 and rebuild tablet_schema + // simulate the case that index properties not equal among input rowsets + // the two cols will skip index compaction and make ctx.columns_to_do_index_compaction empty + TabletSchemaSPtr mock_schema = std::make_shared(); + TabletSchemaPB schema_pb; + _tablet_schema->to_schema_pb(&schema_pb); + auto* index_pb_1 = schema_pb.mutable_index(1); + (*index_pb_1->mutable_properties())["mock_key"] = "mock_value"; + auto* index_pb_2 = schema_pb.mutable_index(2); + (*index_pb_2->mutable_properties())["mock_key"] = "mock_value"; + mock_schema->init_from_pb(schema_pb); + + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + // none index will do index compaction + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 0); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + // set mock_schema to the first input rowset + rowsets[0]->_schema = mock_schema; + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); + + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + // check index file + auto dir_idx_compaction_1 = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(dir_idx_compaction_1.has_value()) << dir_idx_compaction_1.error(); + + // check index 10001 term stats + std::ostringstream oss; + IndexCompactionUtils::check_terms_stats(dir_idx_compaction_1.value().get(), oss); + std::string output = oss.str(); + EXPECT_EQ(output, expected_output); + + auto dir_idx_compaction_2 = inverted_index_file_reader_index->_open(10002, ""); + EXPECT_TRUE(dir_idx_compaction_2.has_value()) << dir_idx_compaction_2.error(); +} + +TEST_F(IndexCompactionTest, test_index_compaction_with_delete) { + EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); + EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); + std::string data_file1 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; + std::string data_file2 = + _current_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; + std::vector data_files; + data_files.push_back(data_file1); + data_files.push_back(data_file2); + + std::vector rowsets(data_files.size()); + auto custom_check_build_rowsets = [](const int32_t& size) { EXPECT_EQ(size, 4); }; + IndexCompactionUtils::build_rowsets(_data_dir, _tablet_schema, _tablet, _engine_ref, rowsets, + data_files, custom_check_build_rowsets); + + // create delete predicate rowset and add to tablet + auto delete_rowset = IndexCompactionUtils::create_delete_predicate_rowset( + _tablet_schema, "v1='great'", inc_id++); + EXPECT_TRUE(_tablet->add_rowset(delete_rowset).ok()); + EXPECT_TRUE(_tablet->rowset_map().size() == 3); + rowsets.push_back(delete_rowset); + EXPECT_TRUE(rowsets.size() == 3); + + auto custom_check_index = [](const BaseCompaction& compaction, const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 2); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_index; + auto st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, true, + output_rowset_index, custom_check_index); + EXPECT_TRUE(st.ok()) << st.to_string(); + + const auto& seg_path = output_rowset_index->segment_path(0); + EXPECT_TRUE(seg_path.has_value()) << seg_path.error(); + auto inverted_index_file_reader_index = IndexCompactionUtils::init_index_file_reader( + output_rowset_index, seg_path.value(), + _tablet_schema->get_inverted_index_storage_format()); + + auto custom_check_normal = [](const BaseCompaction& compaction, + const RowsetWriterContext& ctx) { + EXPECT_EQ(compaction._cur_tablet_schema->inverted_indexes().size(), 4); + EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 0); + EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); + }; + + RowsetSharedPtr output_rowset_normal; + st = IndexCompactionUtils::do_compaction(rowsets, _engine_ref, _tablet, false, + output_rowset_normal, custom_check_normal); + EXPECT_TRUE(st.ok()) << st.to_string(); + const auto& seg_path_normal = output_rowset_normal->segment_path(0); + EXPECT_TRUE(seg_path_normal.has_value()) << seg_path_normal.error(); + auto inverted_index_file_reader_normal = IndexCompactionUtils::init_index_file_reader( + output_rowset_normal, seg_path_normal.value(), + _tablet_schema->get_inverted_index_storage_format()); + // check index file terms + auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); + EXPECT_TRUE(dir_idx_compaction.has_value()) << dir_idx_compaction.error(); + auto dir_normal_compaction = inverted_index_file_reader_normal->_open(10001, ""); + EXPECT_TRUE(dir_normal_compaction.has_value()) << dir_normal_compaction.error(); + std::ostringstream oss; + IndexCompactionUtils::check_terms_stats(dir_idx_compaction->get(), oss); + std::string output = oss.str(); + EXPECT_EQ(output, expected_delete_output); + oss.str(""); + oss.clear(); + IndexCompactionUtils::check_terms_stats(dir_normal_compaction->get(), oss); + output = oss.str(); + EXPECT_EQ(output, expected_delete_output); + + st = IndexCompactionUtils::check_idx_file_correctness(dir_idx_compaction->get(), + dir_normal_compaction->get()); + EXPECT_TRUE(st.ok()) << st.to_string(); + + // check meta and file + std::map query_map = { + {0, {{"99", "66", "56", "87", "85", "96", "20000"}, {19, 21, 21, 16, 14, 18, 0}}}, + {3, {{"99", "66", "56", "87", "85", "96", "10000"}, {12, 18, 22, 21, 16, 20, 0}}}, + {1, {{"good", "maybe", "great", "null"}, {197, 191, 0, 0}}}, + {2, {{"musicstream.com", "http", "https", "null"}, {176, 719, 1087, 0}}}}; + IndexCompactionUtils::check_meta_and_file(output_rowset_index, _tablet_schema, query_map); + IndexCompactionUtils::check_meta_and_file(output_rowset_normal, _tablet_schema, query_map); } } // namespace doris diff --git a/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_with_deleted_term.cpp b/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_with_deleted_term.cpp deleted file mode 100644 index a46f5f210dfc15..00000000000000 --- a/be/test/olap/rowset/segment_v2/inverted_index/compaction/index_compaction_with_deleted_term.cpp +++ /dev/null @@ -1,671 +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 - -#include -#include - -#include "CLucene/StdHeader.h" -#include "CLucene/config/repl_wchar.h" -#include "json2pb/json_to_pb.h" -#include "json2pb/pb_to_json.h" -#include "olap/base_compaction.h" -#include "olap/rowset/beta_rowset.h" -#include "olap/rowset/beta_rowset_writer.h" -#include "olap/rowset/rowset_factory.h" -#include "olap/rowset/segment_v2/inverted_index/query/query_factory.h" -#include "olap/rowset/segment_v2/inverted_index_file_reader.h" -#include "olap/storage_engine.h" - -namespace doris { - -using namespace doris::vectorized; - -constexpr static uint32_t MAX_PATH_LEN = 1024; -constexpr static std::string_view dest_dir = "/ut_dir/inverted_index_test"; -constexpr static std::string_view tmp_dir = "./ut_dir/tmp"; -static int64_t inc_id = 1000; - -struct DataRow { - int key; - std::string word; - std::string url; - int num; -}; - -static std::vector read_data(const std::string file_name) { - std::ifstream file(file_name); - EXPECT_TRUE(file.is_open()); - - std::string line; - std::vector data; - - while (std::getline(file, line)) { - std::stringstream ss(line); - std::string item; - DataRow row; - EXPECT_TRUE(std::getline(ss, item, ',')); - row.key = std::stoi(item); - EXPECT_TRUE(std::getline(ss, item, ',')); - row.word = item; - EXPECT_TRUE(std::getline(ss, item, ',')); - row.url = item; - EXPECT_TRUE(std::getline(ss, item, ',')); - row.num = std::stoi(item); - data.emplace_back(std::move(row)); - } - - file.close(); - return data; -} - -static bool query_bkd(const TabletIndex* index, - std::shared_ptr& inverted_index_file_reader, - const std::vector& query_data, const std::vector& query_result) { - const auto& idx_reader = BkdIndexReader::create_shared(index, inverted_index_file_reader); - const auto& index_searcher_builder = std::make_unique(); - auto dir = inverted_index_file_reader->open(index); - EXPECT_TRUE(dir.has_value()); - auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); - EXPECT_TRUE(searcher_result.has_value()); - auto bkd_searcher = std::get_if(&searcher_result.value()); - EXPECT_TRUE(bkd_searcher != nullptr); - idx_reader->_type_info = get_scalar_type_info((FieldType)(*bkd_searcher)->type); - EXPECT_TRUE(idx_reader->_type_info != nullptr); - idx_reader->_value_key_coder = get_key_coder(idx_reader->_type_info->type()); - - for (int i = 0; i < query_data.size(); i++) { - vectorized::Field param_value = Int32(query_data[i]); - std::unique_ptr query_param = nullptr; - EXPECT_TRUE(segment_v2::InvertedIndexQueryParamFactory::create_query_value( - PrimitiveType::TYPE_INT, ¶m_value, query_param) - .ok()); - auto result = std::make_shared(); - EXPECT_TRUE(idx_reader - ->invoke_bkd_query(query_param->get_value(), - InvertedIndexQueryType::EQUAL_QUERY, *bkd_searcher, - result) - .ok()); - EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; - } - return true; -} - -static bool query_string(const TabletIndex* index, - std::shared_ptr& inverted_index_file_reader, - const std::string& column_name, const std::vector& query_data, - const std::vector& query_result) { - const auto& idx_reader = - StringTypeInvertedIndexReader::create_shared(index, inverted_index_file_reader); - const auto& index_searcher_builder = std::make_unique(); - auto dir = inverted_index_file_reader->open(index); - EXPECT_TRUE(dir.has_value()); - auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); - EXPECT_TRUE(searcher_result.has_value()); - auto string_searcher = std::get_if(&searcher_result.value()); - EXPECT_TRUE(string_searcher != nullptr); - std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); - - for (int i = 0; i < query_data.size(); i++) { - TQueryOptions queryOptions; - auto query = QueryFactory::create(InvertedIndexQueryType::EQUAL_QUERY, *string_searcher, - queryOptions, nullptr); - EXPECT_TRUE(query != nullptr); - InvertedIndexQueryInfo query_info; - query_info.field_name = column_name_ws; - query_info.terms.emplace_back(query_data[i]); - query->add(query_info); - auto result = std::make_shared(); - query->search(*result); - EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; - } - return true; -} - -static bool query_fulltext(const TabletIndex* index, - std::shared_ptr& inverted_index_file_reader, - const std::string& column_name, - const std::vector& query_data, - const std::vector& query_result) { - const auto& idx_reader = FullTextIndexReader::create_shared(index, inverted_index_file_reader); - const auto& index_searcher_builder = std::make_unique(); - auto dir = inverted_index_file_reader->open(index); - EXPECT_TRUE(dir.has_value()); - auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); - EXPECT_TRUE(searcher_result.has_value()); - auto string_searcher = std::get_if(&searcher_result.value()); - EXPECT_TRUE(string_searcher != nullptr); - std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); - - for (int i = 0; i < query_data.size(); i++) { - TQueryOptions queryOptions; - auto query = QueryFactory::create(InvertedIndexQueryType::MATCH_ANY_QUERY, *string_searcher, - queryOptions, nullptr); - EXPECT_TRUE(query != nullptr); - InvertedIndexQueryInfo query_info; - query_info.field_name = column_name_ws; - query_info.terms.emplace_back(query_data[i]); - query->add(query_info); - auto result = std::make_shared(); - query->search(*result); - EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; - } - return true; -} - -static void check_terms_stats(lucene::store::Directory* dir) { - IndexReader* r = IndexReader::open(dir); - - printf("Max Docs: %d\n", r->maxDoc()); - printf("Num Docs: %d\n", r->numDocs()); - - int64_t ver = r->getCurrentVersion(dir); - printf("Current Version: %f\n", (float_t)ver); - - TermEnum* te = r->terms(); - int32_t nterms; - for (nterms = 0; te->next(); nterms++) { - /* empty */ - std::string token = - lucene_wcstoutf8string(te->term(false)->text(), te->term(false)->textLength()); - std::string field = lucene_wcstoutf8string(te->term(false)->field(), - lenOfString(te->term(false)->field())); - - printf("Field: %s ", field.c_str()); - printf("Term: %s ", token.c_str()); - printf("Freq: %d\n", te->docFreq()); - if (false) { - TermDocs* td = r->termDocs(te->term()); - while (td->next()) { - printf("DocID: %d ", td->doc()); - printf("TermFreq: %d\n", td->freq()); - } - _CLLDELETE(td); - } - } - printf("Term count: %d\n\n", nterms); - te->close(); - _CLLDELETE(te); - - r->close(); - _CLLDELETE(r); -} -static Status check_idx_file_correctness(lucene::store::Directory* index_reader, - lucene::store::Directory* tmp_index_reader) { - lucene::index::IndexReader* idx_reader = lucene::index::IndexReader::open(index_reader); - lucene::index::IndexReader* tmp_idx_reader = lucene::index::IndexReader::open(tmp_index_reader); - - // compare numDocs - if (idx_reader->numDocs() != tmp_idx_reader->numDocs()) { - return Status::InternalError( - "index compaction correctness check failed, numDocs not equal, idx_numDocs={}, " - "tmp_idx_numDocs={}", - idx_reader->numDocs(), tmp_idx_reader->numDocs()); - } - - lucene::index::TermEnum* term_enum = idx_reader->terms(); - lucene::index::TermEnum* tmp_term_enum = tmp_idx_reader->terms(); - lucene::index::TermDocs* term_docs = nullptr; - lucene::index::TermDocs* tmp_term_docs = nullptr; - - // iterate TermEnum - while (term_enum->next() && tmp_term_enum->next()) { - std::string token = lucene_wcstoutf8string(term_enum->term(false)->text(), - term_enum->term(false)->textLength()); - std::string field = lucene_wcstoutf8string(term_enum->term(false)->field(), - lenOfString(term_enum->term(false)->field())); - std::string tmp_token = lucene_wcstoutf8string(tmp_term_enum->term(false)->text(), - tmp_term_enum->term(false)->textLength()); - std::string tmp_field = - lucene_wcstoutf8string(tmp_term_enum->term(false)->field(), - lenOfString(tmp_term_enum->term(false)->field())); - // compare token and field - if (field != tmp_field) { - return Status::InternalError( - "index compaction correctness check failed, fields not equal, field={}, " - "tmp_field={}", - field, field); - } - if (token != tmp_token) { - return Status::InternalError( - "index compaction correctness check failed, tokens not equal, token={}, " - "tmp_token={}", - token, tmp_token); - } - - // get term's docId and freq - term_docs = idx_reader->termDocs(term_enum->term(false)); - tmp_term_docs = tmp_idx_reader->termDocs(tmp_term_enum->term(false)); - - // compare term's docId and freq - while (term_docs->next() && tmp_term_docs->next()) { - if (term_docs->doc() != tmp_term_docs->doc() || - term_docs->freq() != tmp_term_docs->freq()) { - return Status::InternalError( - "index compaction correctness check failed, docId or freq not equal, " - "docId={}, tmp_docId={}, freq={}, tmp_freq={}", - term_docs->doc(), tmp_term_docs->doc(), term_docs->freq(), - tmp_term_docs->freq()); - } - } - - // check if there are remaining docs - if (term_docs->next() || tmp_term_docs->next()) { - return Status::InternalError( - "index compaction correctness check failed, number of docs not equal for " - "term={}, tmp_term={}", - token, tmp_token); - } - if (term_docs) { - term_docs->close(); - _CLLDELETE(term_docs); - } - if (tmp_term_docs) { - tmp_term_docs->close(); - _CLLDELETE(tmp_term_docs); - } - } - - // check if there are remaining terms - if (term_enum->next() || tmp_term_enum->next()) { - return Status::InternalError( - "index compaction correctness check failed, number of terms not equal"); - } - if (term_enum) { - term_enum->close(); - _CLLDELETE(term_enum); - } - if (tmp_term_enum) { - tmp_term_enum->close(); - _CLLDELETE(tmp_term_enum); - } - if (idx_reader) { - idx_reader->close(); - _CLLDELETE(idx_reader); - } - if (tmp_idx_reader) { - tmp_idx_reader->close(); - _CLLDELETE(tmp_idx_reader); - } - return Status::OK(); -} - -static RowsetSharedPtr do_compaction(std::vector rowsets, - StorageEngine* engine_ref, TabletSharedPtr tablet, - bool is_index_compaction) { - config::inverted_index_compaction_enable = is_index_compaction; - // only base compaction can handle delete predicate - BaseCompaction compaction(*engine_ref, tablet); - compaction._input_rowsets = std::move(rowsets); - compaction.build_basic_info(); - - std::vector input_rs_readers; - input_rs_readers.reserve(compaction._input_rowsets.size()); - for (auto& rowset : compaction._input_rowsets) { - RowsetReaderSharedPtr rs_reader; - EXPECT_TRUE(rowset->create_reader(&rs_reader).ok()); - input_rs_readers.push_back(std::move(rs_reader)); - } - - RowsetWriterContext ctx; - EXPECT_TRUE(compaction.construct_output_rowset_writer(ctx).ok()); - - if (is_index_compaction) { - EXPECT_TRUE(ctx.columns_to_do_index_compaction.size() == 2); - // col v1 - EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(1)); - // col v2 - EXPECT_TRUE(ctx.columns_to_do_index_compaction.contains(2)); - } - - compaction._stats.rowid_conversion = compaction._rowid_conversion.get(); - EXPECT_TRUE(Merger::vertical_merge_rowsets(tablet, compaction.compaction_type(), - *(compaction._cur_tablet_schema), input_rs_readers, - compaction._output_rs_writer.get(), 100000, 5, - &compaction._stats) - .ok()); - const auto& dst_writer = - dynamic_cast(compaction._output_rs_writer.get()); - for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { - EXPECT_FALSE(idx_file_writer->_closed); - } - Status st = compaction.do_inverted_index_compaction(); - EXPECT_TRUE(st.ok()) << st.to_string(); - - st = compaction._output_rs_writer->build(compaction._output_rowset); - EXPECT_TRUE(st.ok()) << st.to_string(); - - for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { - EXPECT_TRUE(idx_file_writer->_closed); - } - EXPECT_TRUE(compaction._output_rowset->num_segments() == 1); - - return compaction._output_rowset; -} - -class IndexCompactionDeleteTest : public ::testing::Test { -protected: - void SetUp() override { - // absolute dir - char buffer[MAX_PATH_LEN]; - EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr); - _curreent_dir = std::string(buffer); - _absolute_dir = _curreent_dir + std::string(dest_dir); - EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_absolute_dir).ok()); - EXPECT_TRUE(io::global_local_filesystem()->create_directory(_absolute_dir).ok()); - - // tmp dir - EXPECT_TRUE(io::global_local_filesystem()->delete_directory(tmp_dir).ok()); - EXPECT_TRUE(io::global_local_filesystem()->create_directory(tmp_dir).ok()); - std::vector paths; - paths.emplace_back(std::string(tmp_dir), 1024000000); - auto tmp_file_dirs = std::make_unique(paths); - Status st = tmp_file_dirs->init(); - EXPECT_TRUE(st.ok()) << st.to_json(); - ExecEnv::GetInstance()->set_tmp_file_dir(std::move(tmp_file_dirs)); - - // storage engine - doris::EngineOptions options; - auto engine = std::make_unique(options); - _engine_ref = engine.get(); - _data_dir = std::make_unique(*_engine_ref, _absolute_dir); - static_cast(_data_dir->update_capacity()); - ExecEnv::GetInstance()->set_storage_engine(std::move(engine)); - - // tablet_schema - TabletSchemaPB schema_pb; - schema_pb.set_keys_type(KeysType::DUP_KEYS); - schema_pb.set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V2); - - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10000, "key_index", 0, - "INT", "key"); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10001, "v1_index", 1, - "STRING", "v1"); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10002, "v2_index", 2, - "STRING", "v2", true); - construct_column(schema_pb.add_column(), schema_pb.add_index(), 10003, "v3_index", 3, "INT", - "v3"); - - _tablet_schema.reset(new TabletSchema); - _tablet_schema->init_from_pb(schema_pb); - - // tablet - TabletMetaSharedPtr tablet_meta(new TabletMeta(_tablet_schema)); - - _tablet.reset(new Tablet(*_engine_ref, tablet_meta, _data_dir.get())); - EXPECT_TRUE(_tablet->init().ok()); - } - void TearDown() override { - EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); - EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_absolute_dir).ok()); - EXPECT_TRUE(io::global_local_filesystem()->delete_directory(tmp_dir).ok()); - _engine_ref = nullptr; - ExecEnv::GetInstance()->set_storage_engine(nullptr); - } - - void init_rs_meta(RowsetMetaSharedPtr& rs_meta, int64_t start, int64_t end) { - std::string json_rowset_meta = R"({ - "rowset_id": 540081, - "tablet_id": 15673, - "partition_id": 10000, - "tablet_schema_hash": 567997577, - "rowset_type": "BETA_ROWSET", - "rowset_state": "VISIBLE", - "empty": false - })"; - RowsetMetaPB rowset_meta_pb; - json2pb::JsonToProtoMessage(json_rowset_meta, &rowset_meta_pb); - rowset_meta_pb.set_start_version(start); - rowset_meta_pb.set_end_version(end); - rs_meta->init_from_pb(rowset_meta_pb); - } - - RowsetSharedPtr create_delete_predicate_rowset(const TabletSchemaSPtr& schema, std::string pred, - int64_t version) { - DeletePredicatePB del_pred; - del_pred.add_sub_predicates(pred); - del_pred.set_version(1); - RowsetMetaSharedPtr rsm(new RowsetMeta()); - init_rs_meta(rsm, version, version); - RowsetId id; - id.init(version); - rsm->set_rowset_id(id); - rsm->set_delete_predicate(std::move(del_pred)); - rsm->set_tablet_schema(schema); - return std::make_shared(schema, rsm, ""); - } - - void construct_column(ColumnPB* column_pb, TabletIndexPB* tablet_index, int64_t index_id, - const std::string& index_name, int32_t col_unique_id, - const std::string& column_type, const std::string& column_name, - bool parser = false) { - column_pb->set_unique_id(col_unique_id); - column_pb->set_name(column_name); - column_pb->set_type(column_type); - column_pb->set_is_key(false); - column_pb->set_is_nullable(true); - tablet_index->set_index_id(index_id); - tablet_index->set_index_name(index_name); - tablet_index->set_index_type(IndexType::INVERTED); - tablet_index->add_col_unique_id(col_unique_id); - if (parser) { - auto* properties = tablet_index->mutable_properties(); - (*properties)[INVERTED_INDEX_PARSER_KEY] = INVERTED_INDEX_PARSER_UNICODE; - } - } - - void check_meta_and_file(RowsetSharedPtr output_rowset) { - CHECK_EQ(output_rowset->num_segments(), 1); - // check rowset meta and file - int seg_id = 0; - // meta - const auto& index_info = output_rowset->_rowset_meta->inverted_index_file_info(seg_id); - EXPECT_TRUE(index_info.has_index_size()); - const auto& fs = output_rowset->_rowset_meta->fs(); - const auto& file_name = fmt::format("{}/{}_{}.idx", output_rowset->tablet_path(), - output_rowset->rowset_id().to_string(), seg_id); - int64_t file_size = 0; - EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); - EXPECT_EQ(index_info.index_size(), file_size); - - // file - const auto& seg_path = output_rowset->segment_path(seg_id); - EXPECT_TRUE(seg_path.has_value()); - const auto& index_file_path_prefix = - InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); - auto inverted_index_file_reader = std::make_shared( - fs, std::string(index_file_path_prefix), - _tablet_schema->get_inverted_index_storage_format(), index_info); - EXPECT_TRUE(inverted_index_file_reader->init().ok()); - const auto& dirs = inverted_index_file_reader->get_all_directories(); - EXPECT_TRUE(dirs.has_value()); - EXPECT_EQ(dirs.value().size(), 4); - - // read col key - const auto& key = _tablet_schema->column_by_uid(0); - const auto* key_index = _tablet_schema->inverted_index(key); - EXPECT_TRUE(key_index != nullptr); - std::vector query_data {99, 66, 56, 87, 85, 96, 20000}; - std::vector query_result {19, 21, 21, 16, 14, 18, 0}; - EXPECT_TRUE(query_bkd(key_index, inverted_index_file_reader, query_data, query_result)); - - // read col v3 - const auto& v3_column = _tablet_schema->column_by_uid(3); - const auto* v3_index = _tablet_schema->inverted_index(v3_column); - EXPECT_TRUE(v3_index != nullptr); - std::vector query_data3 {99, 66, 56, 87, 85, 96, 10000}; - std::vector query_result3 {12, 18, 22, 21, 16, 20, 0}; - EXPECT_TRUE(query_bkd(v3_index, inverted_index_file_reader, query_data3, query_result3)); - - // read col v1 - const auto& v1_column = _tablet_schema->column_by_uid(1); - const auto* v1_index = _tablet_schema->inverted_index(v1_column); - EXPECT_TRUE(v1_index != nullptr); - std::vector query_data1 {"good", "maybe", "great", "null"}; - std::vector query_result1 {197, 191, 0, 0}; - EXPECT_TRUE(query_string(v1_index, inverted_index_file_reader, "1", query_data1, - query_result1)); - - // read col v2 - const auto& v2_column = _tablet_schema->column_by_uid(2); - const auto* v2_index = _tablet_schema->inverted_index(v2_column); - EXPECT_TRUE(v2_index != nullptr); - std::vector query_data2 {"musicstream.com", "http", "https", "null"}; - std::vector query_result2 {176, 719, 1087, 0}; - EXPECT_TRUE(query_fulltext(v2_index, inverted_index_file_reader, "2", query_data2, - query_result2)); - } - - RowsetWriterContext rowset_writer_context() { - RowsetWriterContext context; - RowsetId rowset_id; - rowset_id.init(inc_id); - context.rowset_id = rowset_id; - context.rowset_type = BETA_ROWSET; - context.data_dir = _data_dir.get(); - context.rowset_state = VISIBLE; - context.tablet_schema = _tablet_schema; - context.tablet_path = _tablet->tablet_path(); - context.version = Version(inc_id, inc_id); - context.max_rows_per_segment = 200; - inc_id++; - return context; - } - - IndexCompactionDeleteTest() = default; - ~IndexCompactionDeleteTest() override = default; - -private: - TabletSchemaSPtr _tablet_schema = nullptr; - StorageEngine* _engine_ref = nullptr; - std::unique_ptr _data_dir = nullptr; - TabletSharedPtr _tablet = nullptr; - std::string _absolute_dir; - std::string _curreent_dir; -}; - -TEST_F(IndexCompactionDeleteTest, delete_index_test) { - EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_tablet->tablet_path()).ok()); - EXPECT_TRUE(io::global_local_filesystem()->create_directory(_tablet->tablet_path()).ok()); - std::string data_file1 = - _curreent_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data1.csv"; - std::string data_file2 = - _curreent_dir + "/be/test/olap/rowset/segment_v2/inverted_index/data/data2.csv"; - - std::vector> data; - data.emplace_back(read_data(data_file1)); - data.emplace_back(read_data(data_file2)); - - std::vector rowsets(data.size()); - for (int i = 0; i < data.size(); i++) { - const auto& res = - RowsetFactory::create_rowset_writer(*_engine_ref, rowset_writer_context(), false); - EXPECT_TRUE(res.has_value()) << res.error(); - const auto& rowset_writer = res.value(); - - Block block = _tablet_schema->create_block(); - auto columns = block.mutate_columns(); - for (const auto& row : data[i]) { - vectorized::Field key = Int32(row.key); - vectorized::Field v1(row.word); - vectorized::Field v2(row.url); - vectorized::Field v3 = Int32(row.num); - columns[0]->insert(key); - columns[1]->insert(v1); - columns[2]->insert(v2); - columns[3]->insert(v3); - } - EXPECT_TRUE(rowset_writer->add_block(&block).ok()); - EXPECT_TRUE(rowset_writer->flush().ok()); - const auto& dst_writer = dynamic_cast(rowset_writer.get()); - - // inverted index file writer - for (const auto& [seg_id, idx_file_writer] : dst_writer->_idx_files.get_file_writers()) { - EXPECT_TRUE(idx_file_writer->_closed); - } - - EXPECT_TRUE(rowset_writer->build(rowsets[i]).ok()); - EXPECT_TRUE(_tablet->add_rowset(rowsets[i]).ok()); - EXPECT_TRUE(rowsets[i]->num_segments() == 5); - - // check rowset meta and file - for (int seg_id = 0; seg_id < rowsets[i]->num_segments(); seg_id++) { - const auto& index_info = rowsets[i]->_rowset_meta->inverted_index_file_info(seg_id); - EXPECT_TRUE(index_info.has_index_size()); - const auto& fs = rowsets[i]->_rowset_meta->fs(); - const auto& file_name = fmt::format("{}/{}_{}.idx", rowsets[i]->tablet_path(), - rowsets[i]->rowset_id().to_string(), seg_id); - int64_t file_size = 0; - EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); - EXPECT_EQ(index_info.index_size(), file_size); - - const auto& seg_path = rowsets[i]->segment_path(seg_id); - EXPECT_TRUE(seg_path.has_value()); - const auto& index_file_path_prefix = - InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); - auto inverted_index_file_reader = std::make_shared( - fs, std::string(index_file_path_prefix), - _tablet_schema->get_inverted_index_storage_format(), index_info); - EXPECT_TRUE(inverted_index_file_reader->init().ok()); - const auto& dirs = inverted_index_file_reader->get_all_directories(); - EXPECT_TRUE(dirs.has_value()); - EXPECT_EQ(dirs.value().size(), 4); - } - } - - // create delete predicate rowset and add to tablet - auto delete_rowset = create_delete_predicate_rowset(_tablet_schema, "v1='great'", inc_id++); - EXPECT_TRUE(_tablet->add_rowset(delete_rowset).ok()); - EXPECT_TRUE(_tablet->rowset_map().size() == 3); - rowsets.push_back(delete_rowset); - EXPECT_TRUE(rowsets.size() == 3); - - auto output_rowset_index = do_compaction(rowsets, _engine_ref, _tablet, true); - const auto& seg_path = output_rowset_index->segment_path(0); - EXPECT_TRUE(seg_path.has_value()); - const auto& index_file_path_prefix = - InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); - auto inverted_index_file_reader_index = std::make_shared( - output_rowset_index->_rowset_meta->fs(), std::string(index_file_path_prefix), - _tablet_schema->get_inverted_index_storage_format()); - EXPECT_TRUE(inverted_index_file_reader_index->init().ok()); - - auto output_rowset_normal = do_compaction(rowsets, _engine_ref, _tablet, false); - const auto& seg_path_normal = output_rowset_normal->segment_path(0); - EXPECT_TRUE(seg_path_normal.has_value()); - const auto& index_file_path_prefix_normal = - InvertedIndexDescriptor::get_index_file_path_prefix(seg_path_normal.value()); - auto inverted_index_file_reader_normal = std::make_shared( - output_rowset_normal->_rowset_meta->fs(), std::string(index_file_path_prefix_normal), - _tablet_schema->get_inverted_index_storage_format()); - EXPECT_TRUE(inverted_index_file_reader_normal->init().ok()); - - // check index file terms - auto dir_idx_compaction = inverted_index_file_reader_index->_open(10001, ""); - auto dir_normal_compaction = inverted_index_file_reader_normal->_open(10001, ""); - check_terms_stats(dir_idx_compaction->get()); - check_terms_stats(dir_normal_compaction->get()); - auto st = check_idx_file_correctness(dir_idx_compaction->get(), dir_normal_compaction->get()); - EXPECT_TRUE(st.ok()) << st.to_string(); - - // check meta and file - check_meta_and_file(output_rowset_index); - check_meta_and_file(output_rowset_normal); -} - -} // namespace doris diff --git a/be/test/olap/rowset/segment_v2/inverted_index/compaction/util/index_compaction_utils.cpp b/be/test/olap/rowset/segment_v2/inverted_index/compaction/util/index_compaction_utils.cpp new file mode 100644 index 00000000000000..530dca8054c19a --- /dev/null +++ b/be/test/olap/rowset/segment_v2/inverted_index/compaction/util/index_compaction_utils.cpp @@ -0,0 +1,621 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include + +#include +#include +#include +#include +#include +#include + +#include "CLucene/StdHeader.h" +#include "CLucene/config/repl_wchar.h" +#include "json2pb/json_to_pb.h" +#include "json2pb/pb_to_json.h" +#include "olap/base_compaction.h" +#include "olap/rowset/beta_rowset.h" +#include "olap/rowset/beta_rowset_writer.h" +#include "olap/rowset/rowset_factory.h" +#include "olap/rowset/segment_v2/inverted_index/query/query_factory.h" +#include "olap/rowset/segment_v2/inverted_index_file_reader.h" +#include "olap/storage_engine.h" + +namespace doris { + +static int64_t inc_id = 1000; +const static std::string expected_output = + "Max Docs: 2000\n" + "Num Docs: 2000\n" + "Field: 1 Term: bad Freq: 196\n" + "Field: 1 Term: excellent Freq: 227\n" + "Field: 1 Term: fine Freq: 190\n" + "Field: 1 Term: good Freq: 197\n" + "Field: 1 Term: great Freq: 194\n" + "Field: 1 Term: maybe Freq: 191\n" + "Field: 1 Term: no Freq: 205\n" + "Field: 1 Term: ok Freq: 175\n" + "Field: 1 Term: terrible Freq: 205\n" + "Field: 1 Term: yes Freq: 220\n" + "Term count: 10\n\n"; +const static std::string expected_delete_output = + "Max Docs: 1806\n" + "Num Docs: 1806\n" + "Field: 1 Term: bad Freq: 196\n" + "Field: 1 Term: excellent Freq: 227\n" + "Field: 1 Term: fine Freq: 190\n" + "Field: 1 Term: good Freq: 197\n" + "Field: 1 Term: maybe Freq: 191\n" + "Field: 1 Term: no Freq: 205\n" + "Field: 1 Term: ok Freq: 175\n" + "Field: 1 Term: terrible Freq: 205\n" + "Field: 1 Term: yes Freq: 220\n" + "Term count: 9\n\n"; + +using QueryData = std::pair, std::vector>; + +class IndexCompactionUtils { + struct DataRow { + int key; + std::string word; + std::string url; + int num; + }; + + static std::vector read_data(const std::string file_name) { + std::ifstream file(file_name); + EXPECT_TRUE(file.is_open()); + + std::string line; + std::vector data; + + while (std::getline(file, line)) { + std::stringstream ss(line); + std::string item; + DataRow row; + EXPECT_TRUE(std::getline(ss, item, ',')); + row.key = std::stoi(item); + EXPECT_TRUE(std::getline(ss, item, ',')); + row.word = item; + EXPECT_TRUE(std::getline(ss, item, ',')); + row.url = item; + EXPECT_TRUE(std::getline(ss, item, ',')); + row.num = std::stoi(item); + data.emplace_back(std::move(row)); + } + + file.close(); + return data; + } + + static bool query_bkd(const TabletIndex* index, + std::shared_ptr& inverted_index_file_reader, + const std::vector& query_data, + const std::vector& query_result) { + const auto& idx_reader = BkdIndexReader::create_shared(index, inverted_index_file_reader); + const auto& index_searcher_builder = std::make_unique(); + auto dir = inverted_index_file_reader->open(index); + EXPECT_TRUE(dir.has_value()); + auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); + EXPECT_TRUE(searcher_result.has_value()); + auto bkd_searcher = std::get_if(&searcher_result.value()); + EXPECT_TRUE(bkd_searcher != nullptr); + idx_reader->_type_info = get_scalar_type_info((FieldType)(*bkd_searcher)->type); + EXPECT_TRUE(idx_reader->_type_info != nullptr); + idx_reader->_value_key_coder = get_key_coder(idx_reader->_type_info->type()); + + for (int i = 0; i < query_data.size(); i++) { + vectorized::Field param_value = int32_t(query_data[i]); + std::unique_ptr query_param = nullptr; + EXPECT_TRUE(segment_v2::InvertedIndexQueryParamFactory::create_query_value( + PrimitiveType::TYPE_INT, ¶m_value, query_param) + .ok()); + auto result = std::make_shared(); + EXPECT_TRUE(idx_reader + ->invoke_bkd_query(query_param->get_value(), + InvertedIndexQueryType::EQUAL_QUERY, + *bkd_searcher, result) + .ok()); + EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; + } + return true; + } + + static bool query_string(const TabletIndex* index, + std::shared_ptr& inverted_index_file_reader, + const std::string& column_name, + const std::vector& query_data, + const std::vector& query_result) { + const auto& idx_reader = + StringTypeInvertedIndexReader::create_shared(index, inverted_index_file_reader); + const auto& index_searcher_builder = std::make_unique(); + auto dir = inverted_index_file_reader->open(index); + EXPECT_TRUE(dir.has_value()); + auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); + EXPECT_TRUE(searcher_result.has_value()); + auto string_searcher = std::get_if(&searcher_result.value()); + EXPECT_TRUE(string_searcher != nullptr); + std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); + + for (int i = 0; i < query_data.size(); i++) { + TQueryOptions queryOptions; + auto query = QueryFactory::create(InvertedIndexQueryType::EQUAL_QUERY, *string_searcher, + queryOptions, nullptr); + EXPECT_TRUE(query != nullptr); + InvertedIndexQueryInfo query_info; + query_info.field_name = column_name_ws; + query_info.terms.emplace_back(query_data[i]); + query->add(query_info); + auto result = std::make_shared(); + query->search(*result); + EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; + } + return true; + } + + static bool query_fulltext(const TabletIndex* index, + std::shared_ptr& inverted_index_file_reader, + const std::string& column_name, + const std::vector& query_data, + const std::vector& query_result) { + const auto& idx_reader = + FullTextIndexReader::create_shared(index, inverted_index_file_reader); + const auto& index_searcher_builder = std::make_unique(); + auto dir = inverted_index_file_reader->open(index); + EXPECT_TRUE(dir.has_value()); + auto searcher_result = index_searcher_builder->get_index_searcher(dir.value().release()); + EXPECT_TRUE(searcher_result.has_value()); + auto string_searcher = std::get_if(&searcher_result.value()); + EXPECT_TRUE(string_searcher != nullptr); + std::wstring column_name_ws = StringUtil::string_to_wstring(column_name); + + for (int i = 0; i < query_data.size(); i++) { + TQueryOptions queryOptions; + auto query = QueryFactory::create(InvertedIndexQueryType::MATCH_ANY_QUERY, + *string_searcher, queryOptions, nullptr); + EXPECT_TRUE(query != nullptr); + InvertedIndexQueryInfo query_info; + query_info.field_name = column_name_ws; + query_info.terms.emplace_back(query_data[i]); + query->add(query_info); + auto result = std::make_shared(); + query->search(*result); + EXPECT_EQ(query_result[i], result->cardinality()) << query_data[i]; + } + return true; + } + + static void check_terms_stats(lucene::store::Directory* dir, std::ostream& os = std::cout) { + IndexReader* r = IndexReader::open(dir); + + os << "Max Docs: " << r->maxDoc() << "\n"; + os << "Num Docs: " << r->numDocs() << "\n"; + + TermEnum* te = r->terms(); + int32_t nterms; + for (nterms = 0; te->next(); nterms++) { + std::string token = + lucene_wcstoutf8string(te->term(false)->text(), te->term(false)->textLength()); + std::string field = lucene_wcstoutf8string(te->term(false)->field(), + lenOfString(te->term(false)->field())); + + os << "Field: " << field << " "; + os << "Term: " << token << " "; + os << "Freq: " << te->docFreq() << "\n"; + if (false) { + TermDocs* td = r->termDocs(te->term()); + while (td->next()) { + os << "DocID: " << td->doc() << " "; + os << "TermFreq: " << td->freq() << "\n"; + } + _CLLDELETE(td); + } + } + os << "Term count: " << nterms << "\n\n"; + te->close(); + _CLLDELETE(te); + + r->close(); + _CLLDELETE(r); + } + static Status check_idx_file_correctness(lucene::store::Directory* index_reader, + lucene::store::Directory* tmp_index_reader) { + lucene::index::IndexReader* idx_reader = lucene::index::IndexReader::open(index_reader); + lucene::index::IndexReader* tmp_idx_reader = + lucene::index::IndexReader::open(tmp_index_reader); + + // compare numDocs + if (idx_reader->numDocs() != tmp_idx_reader->numDocs()) { + return Status::InternalError( + "index compaction correctness check failed, numDocs not equal, idx_numDocs={}, " + "tmp_idx_numDocs={}", + idx_reader->numDocs(), tmp_idx_reader->numDocs()); + } + + lucene::index::TermEnum* term_enum = idx_reader->terms(); + lucene::index::TermEnum* tmp_term_enum = tmp_idx_reader->terms(); + lucene::index::TermDocs* term_docs = nullptr; + lucene::index::TermDocs* tmp_term_docs = nullptr; + + // iterate TermEnum + while (term_enum->next() && tmp_term_enum->next()) { + std::string token = lucene_wcstoutf8string(term_enum->term(false)->text(), + term_enum->term(false)->textLength()); + std::string field = lucene_wcstoutf8string( + term_enum->term(false)->field(), lenOfString(term_enum->term(false)->field())); + std::string tmp_token = lucene_wcstoutf8string( + tmp_term_enum->term(false)->text(), tmp_term_enum->term(false)->textLength()); + std::string tmp_field = + lucene_wcstoutf8string(tmp_term_enum->term(false)->field(), + lenOfString(tmp_term_enum->term(false)->field())); + // compare token and field + if (field != tmp_field) { + return Status::InternalError( + "index compaction correctness check failed, fields not equal, field={}, " + "tmp_field={}", + field, field); + } + if (token != tmp_token) { + return Status::InternalError( + "index compaction correctness check failed, tokens not equal, token={}, " + "tmp_token={}", + token, tmp_token); + } + + // get term's docId and freq + term_docs = idx_reader->termDocs(term_enum->term(false)); + tmp_term_docs = tmp_idx_reader->termDocs(tmp_term_enum->term(false)); + + // compare term's docId and freq + while (term_docs->next() && tmp_term_docs->next()) { + if (term_docs->doc() != tmp_term_docs->doc() || + term_docs->freq() != tmp_term_docs->freq()) { + return Status::InternalError( + "index compaction correctness check failed, docId or freq not equal, " + "docId={}, tmp_docId={}, freq={}, tmp_freq={}", + term_docs->doc(), tmp_term_docs->doc(), term_docs->freq(), + tmp_term_docs->freq()); + } + } + + // check if there are remaining docs + if (term_docs->next() || tmp_term_docs->next()) { + return Status::InternalError( + "index compaction correctness check failed, number of docs not equal for " + "term={}, tmp_term={}", + token, tmp_token); + } + if (term_docs) { + term_docs->close(); + _CLLDELETE(term_docs); + } + if (tmp_term_docs) { + tmp_term_docs->close(); + _CLLDELETE(tmp_term_docs); + } + } + + // check if there are remaining terms + if (term_enum->next() || tmp_term_enum->next()) { + return Status::InternalError( + "index compaction correctness check failed, number of terms not equal"); + } + if (term_enum) { + term_enum->close(); + _CLLDELETE(term_enum); + } + if (tmp_term_enum) { + tmp_term_enum->close(); + _CLLDELETE(tmp_term_enum); + } + if (idx_reader) { + idx_reader->close(); + _CLLDELETE(idx_reader); + } + if (tmp_idx_reader) { + tmp_idx_reader->close(); + _CLLDELETE(tmp_idx_reader); + } + return Status::OK(); + } + + static Status do_compaction( + const std::vector& rowsets, StorageEngine* engine_ref, + const TabletSharedPtr& tablet, bool is_index_compaction, RowsetSharedPtr& rowset_ptr, + const std::function + custom_check = nullptr) { + config::inverted_index_compaction_enable = is_index_compaction; + // only base compaction can handle delete predicate + BaseCompaction compaction(*engine_ref, tablet); + compaction._input_rowsets = std::move(rowsets); + compaction.build_basic_info(); + + std::vector input_rs_readers; + create_input_rowsets_readers(compaction, input_rs_readers); + + RowsetWriterContext ctx; + RETURN_IF_ERROR(compaction.construct_output_rowset_writer(ctx)); + + compaction._stats.rowid_conversion = compaction._rowid_conversion.get(); + RETURN_IF_ERROR(Merger::vertical_merge_rowsets( + tablet, compaction.compaction_type(), *(compaction._cur_tablet_schema), + input_rs_readers, compaction._output_rs_writer.get(), 100000, 5, + &compaction._stats)); + + const auto& dst_writer = + dynamic_cast(compaction._output_rs_writer.get()); + check_idx_file_writer_closed(dst_writer, false); + + RETURN_IF_ERROR(compaction.do_inverted_index_compaction()); + + RETURN_IF_ERROR(compaction._output_rs_writer->build(compaction._output_rowset)); + + check_idx_file_writer_closed(dst_writer, true); + + if (custom_check) { + custom_check(compaction, ctx); + } + + rowset_ptr = std::move(compaction._output_rowset); + return Status::OK(); + } + + static void create_input_rowsets_readers(const BaseCompaction& compaction, + std::vector& input_rs_readers) { + input_rs_readers.reserve(compaction._input_rowsets.size()); + for (auto& rowset : compaction._input_rowsets) { + RowsetReaderSharedPtr rs_reader; + EXPECT_TRUE(rowset->create_reader(&rs_reader).ok()); + input_rs_readers.push_back(std::move(rs_reader)); + } + } + + static void check_idx_file_writer_closed(BaseBetaRowsetWriter* writer, bool closed) { + for (const auto& [seg_id, idx_file_writer] : writer->inverted_index_file_writers()) { + EXPECT_EQ(idx_file_writer->_closed, closed); + } + } + + static void init_rs_meta(RowsetMetaSharedPtr& rs_meta, int64_t start, int64_t end) { + std::string json_rowset_meta = R"({ + "rowset_id": 540081, + "tablet_id": 15673, + "partition_id": 10000, + "tablet_schema_hash": 567997577, + "rowset_type": "BETA_ROWSET", + "rowset_state": "VISIBLE", + "empty": false + })"; + RowsetMetaPB rowset_meta_pb; + json2pb::JsonToProtoMessage(json_rowset_meta, &rowset_meta_pb); + rowset_meta_pb.set_start_version(start); + rowset_meta_pb.set_end_version(end); + rs_meta->init_from_pb(rowset_meta_pb); + } + + static RowsetSharedPtr create_delete_predicate_rowset(const TabletSchemaSPtr& schema, + std::string pred, int64_t version) { + DeletePredicatePB del_pred; + del_pred.add_sub_predicates(pred); + del_pred.set_version(1); + RowsetMetaSharedPtr rsm(new RowsetMeta()); + init_rs_meta(rsm, version, version); + RowsetId id; + id.init(version); + rsm->set_rowset_id(id); + rsm->set_delete_predicate(std::move(del_pred)); + rsm->set_tablet_schema(schema); + return std::make_shared(schema, rsm, ""); + } + + static void construct_column(ColumnPB* column_pb, TabletIndexPB* tablet_index, int64_t index_id, + const std::string& index_name, int32_t col_unique_id, + const std::string& column_type, const std::string& column_name, + bool parser = false) { + column_pb->set_unique_id(col_unique_id); + column_pb->set_name(column_name); + column_pb->set_type(column_type); + column_pb->set_is_key(false); + column_pb->set_is_nullable(true); + tablet_index->set_index_id(index_id); + tablet_index->set_index_name(index_name); + tablet_index->set_index_type(IndexType::INVERTED); + tablet_index->add_col_unique_id(col_unique_id); + if (parser) { + auto* properties = tablet_index->mutable_properties(); + (*properties)[INVERTED_INDEX_PARSER_KEY] = INVERTED_INDEX_PARSER_UNICODE; + } + } + + static void construct_column(ColumnPB* column_pb, int32_t col_unique_id, + const std::string& column_type, const std::string& column_name) { + column_pb->set_unique_id(col_unique_id); + column_pb->set_name(column_name); + column_pb->set_type(column_type); + column_pb->set_is_key(false); + column_pb->set_is_nullable(true); + } + + static void construct_index(TabletIndexPB* tablet_index, int64_t index_id, + const std::string& index_name, int32_t col_unique_id, + bool parser = false) { + tablet_index->set_index_id(index_id); + tablet_index->set_index_name(index_name); + tablet_index->set_index_type(IndexType::INVERTED); + tablet_index->add_col_unique_id(col_unique_id); + if (parser) { + auto* properties = tablet_index->mutable_properties(); + (*properties)[INVERTED_INDEX_PARSER_KEY] = INVERTED_INDEX_PARSER_UNICODE; + } + } + + static void check_meta_and_file(const RowsetSharedPtr& output_rowset, + const TabletSchemaSPtr& tablet_schema, + const std::map& query_map) { + CHECK_EQ(output_rowset->num_segments(), 1); + // check rowset meta and file + int seg_id = 0; + // meta + const auto& index_info = output_rowset->_rowset_meta->inverted_index_file_info(seg_id); + EXPECT_TRUE(index_info.has_index_size()); + const auto& fs = output_rowset->_rowset_meta->fs(); + const auto& file_name = fmt::format("{}/{}_{}.idx", output_rowset->tablet_path(), + output_rowset->rowset_id().to_string(), seg_id); + int64_t file_size = 0; + EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); + EXPECT_EQ(index_info.index_size(), file_size); + + // file + const auto& seg_path = output_rowset->segment_path(seg_id); + EXPECT_TRUE(seg_path.has_value()); + const auto& index_file_path_prefix = + InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); + auto inverted_index_file_reader = std::make_shared( + fs, std::string(index_file_path_prefix), + tablet_schema->get_inverted_index_storage_format(), index_info); + EXPECT_TRUE(inverted_index_file_reader->init().ok()); + const auto& dirs = inverted_index_file_reader->get_all_directories(); + EXPECT_TRUE(dirs.has_value()); + EXPECT_EQ(dirs.value().size(), 4); + + for (const auto& [col_uid, query_data] : query_map) { + const auto& column = tablet_schema->column_by_uid(col_uid); + const auto* index = tablet_schema->inverted_index(column); + EXPECT_TRUE(index != nullptr); + + if (col_uid == 0 || col_uid == 3) { + // BKD index + std::vector query_data_int; + for (const auto& data : query_data.first) { + query_data_int.push_back(std::stoi(data)); + } + EXPECT_TRUE(query_bkd(index, inverted_index_file_reader, query_data_int, + query_data.second)); + } else if (col_uid == 1) { + // String index + EXPECT_TRUE(query_string(index, inverted_index_file_reader, std::to_string(col_uid), + query_data.first, query_data.second)); + } else if (col_uid == 2) { + // Fulltext index + EXPECT_TRUE(query_fulltext(index, inverted_index_file_reader, + std::to_string(col_uid), query_data.first, + query_data.second)); + } + } + } + + static RowsetWriterContext rowset_writer_context(const std::unique_ptr& data_dir, + const TabletSchemaSPtr& schema, + const std::string& tablet_path) { + RowsetWriterContext context; + RowsetId rowset_id; + rowset_id.init(inc_id); + context.rowset_id = rowset_id; + context.rowset_type = BETA_ROWSET; + context.data_dir = data_dir.get(); + context.rowset_state = VISIBLE; + context.tablet_schema = schema; + context.tablet_path = tablet_path; + context.version = Version(inc_id, inc_id); + context.max_rows_per_segment = 200; + inc_id++; + return context; + } + + static void build_rowsets(const std::unique_ptr& data_dir, + const TabletSchemaSPtr& schema, const TabletSharedPtr& tablet, + StorageEngine* engine_ref, std::vector& rowsets, + const std::vector& data_files, + const std::function custom_check = nullptr) { + std::vector> data; + for (auto file : data_files) { + data.emplace_back(read_data(file)); + } + for (int i = 0; i < data.size(); i++) { + const auto& res = RowsetFactory::create_rowset_writer( + *engine_ref, rowset_writer_context(data_dir, schema, tablet->tablet_path()), + false); + EXPECT_TRUE(res.has_value()) << res.error(); + const auto& rowset_writer = res.value(); + + vectorized::Block block = schema->create_block(); + auto columns = block.mutate_columns(); + for (const auto& row : data[i]) { + vectorized::Field key = int32_t(row.key); + vectorized::Field v1(row.word); + vectorized::Field v2(row.url); + vectorized::Field v3 = int32_t(row.num); + columns[0]->insert(key); + columns[1]->insert(v1); + columns[2]->insert(v2); + columns[3]->insert(v3); + } + EXPECT_TRUE(rowset_writer->add_block(&block).ok()); + EXPECT_TRUE(rowset_writer->flush().ok()); + const auto& dst_writer = dynamic_cast(rowset_writer.get()); + + check_idx_file_writer_closed(dst_writer, true); + + EXPECT_TRUE(rowset_writer->build(rowsets[i]).ok()); + EXPECT_TRUE(tablet->add_rowset(rowsets[i]).ok()); + EXPECT_TRUE(rowsets[i]->num_segments() == 5); + + // check rowset meta and file + for (int seg_id = 0; seg_id < rowsets[i]->num_segments(); seg_id++) { + const auto& index_info = rowsets[i]->_rowset_meta->inverted_index_file_info(seg_id); + EXPECT_TRUE(index_info.has_index_size()); + const auto& fs = rowsets[i]->_rowset_meta->fs(); + const auto& file_name = fmt::format("{}/{}_{}.idx", rowsets[i]->tablet_path(), + rowsets[i]->rowset_id().to_string(), seg_id); + int64_t file_size = 0; + EXPECT_TRUE(fs->file_size(file_name, &file_size).ok()); + EXPECT_EQ(index_info.index_size(), file_size); + + const auto& seg_path = rowsets[i]->segment_path(seg_id); + EXPECT_TRUE(seg_path.has_value()); + const auto& index_file_path_prefix = + InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value()); + auto inverted_index_file_reader = std::make_shared( + fs, std::string(index_file_path_prefix), + schema->get_inverted_index_storage_format(), index_info); + EXPECT_TRUE(inverted_index_file_reader->init().ok()); + const auto& dirs = inverted_index_file_reader->get_all_directories(); + EXPECT_TRUE(dirs.has_value()); + if (custom_check) { + custom_check(dirs.value().size()); + } + } + } + } + + static std::shared_ptr init_index_file_reader( + const RowsetSharedPtr& output_rowset, const std::string& seg_path, + const InvertedIndexStorageFormatPB& index_storage_format) { + const auto& index_file_path_prefix = + InvertedIndexDescriptor::get_index_file_path_prefix(seg_path); + auto inverted_index_file_reader_index = std::make_shared( + output_rowset->_rowset_meta->fs(), std::string(index_file_path_prefix), + index_storage_format); + auto st = inverted_index_file_reader_index->init(); + EXPECT_TRUE(st.ok()) << st.to_string(); + + return inverted_index_file_reader_index; + } +}; + +} // namespace doris \ No newline at end of file diff --git a/be/test/runtime/memory/mem_tracker_test.cpp b/be/test/runtime/memory/mem_tracker_test.cpp index 49f6aa3bf0cebe..eb66635ce072d2 100644 --- a/be/test/runtime/memory/mem_tracker_test.cpp +++ b/be/test/runtime/memory/mem_tracker_test.cpp @@ -26,7 +26,7 @@ namespace doris { TEST(MemTrackerTest, SingleTrackerNoLimit) { - auto t = MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL); + auto t = MemTrackerLimiter::create_shared(MemTrackerLimiter::Type::GLOBAL, "UT"); EXPECT_FALSE(t->has_limit()); t->consume(10); EXPECT_EQ(t->consumption(), 10); diff --git a/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp b/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp index fad2116fca7630..b0974460f4539c 100644 --- a/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp +++ b/be/test/runtime/memory/thread_mem_tracker_mgr_test.cpp @@ -231,7 +231,7 @@ TEST_F(ThreadMemTrackerMgrTest, ReserveMemory) { int64_t size1 = 4 * 1024; int64_t size2 = 4 * 1024 * 1024; - int64_t size3 = size2 * 1024; + int64_t size3 = size2 * 2; thread_context->attach_task(TUniqueId(), t, workload_group); thread_context->consume_memory(size1); @@ -255,7 +255,7 @@ TEST_F(ThreadMemTrackerMgrTest, ReserveMemory) { // std::abs(-size1 - size1) < SYNC_PROC_RESERVED_INTERVAL_BYTES, not update process_reserved_memory. EXPECT_EQ(doris::GlobalMemoryArbitrator::process_reserved_memory(), size3 - size2); - thread_context->consume_memory(size2 * 1023); + thread_context->consume_memory(size2); EXPECT_EQ(t->consumption(), size1 + size2 + size3); EXPECT_EQ(doris::GlobalMemoryArbitrator::process_reserved_memory(), size1 + size1); @@ -325,7 +325,7 @@ TEST_F(ThreadMemTrackerMgrTest, NestedReserveMemory) { MemTrackerLimiter::Type::OTHER, "UT-NestedReserveMemory"); int64_t size2 = 4 * 1024 * 1024; - int64_t size3 = size2 * 1024; + int64_t size3 = size2 * 2; thread_context->attach_task(TUniqueId(), t, workload_group); auto st = thread_context->try_reserve_memory(size3); @@ -382,7 +382,7 @@ TEST_F(ThreadMemTrackerMgrTest, NestedSwitchMemTrackerReserveMemory) { int64_t size1 = 4 * 1024; int64_t size2 = 4 * 1024 * 1024; - int64_t size3 = size2 * 1024; + int64_t size3 = size2 * 2; thread_context->attach_task(TUniqueId(), t1, workload_group); auto st = thread_context->try_reserve_memory(size3); diff --git a/be/test/testutil/run_all_tests.cpp b/be/test/testutil/run_all_tests.cpp index 70af1d2e34f2df..59933db80e5bb9 100644 --- a/be/test/testutil/run_all_tests.cpp +++ b/be/test/testutil/run_all_tests.cpp @@ -56,6 +56,8 @@ int main(int argc, char** argv) { "BE-UT"); doris::thread_context()->thread_mem_tracker_mgr->attach_limiter_tracker(test_tracker); doris::ExecEnv::GetInstance()->set_cache_manager(doris::CacheManager::create_global_instance()); + doris::ExecEnv::GetInstance()->set_process_profile( + doris::ProcessProfile::create_global_instance()); doris::ExecEnv::GetInstance()->set_dummy_lru_cache(std::make_shared()); doris::ExecEnv::GetInstance()->set_storage_page_cache( doris::StoragePageCache::create_global_cache(1 << 30, 10, 0)); diff --git a/build.sh b/build.sh index 6f3ddfa236fb10..8f1262aa76b211 100755 --- a/build.sh +++ b/build.sh @@ -525,6 +525,7 @@ fi if [[ "${BUILD_BE_JAVA_EXTENSIONS}" -eq 1 ]]; then modules+=("fe-common") modules+=("be-java-extensions/hudi-scanner") + modules+=("be-java-extensions/hadoop-hudi-scanner") modules+=("be-java-extensions/java-common") modules+=("be-java-extensions/java-udf") modules+=("be-java-extensions/jdbc-scanner") @@ -814,6 +815,7 @@ EOF extensions_modules=("java-udf") extensions_modules+=("jdbc-scanner") extensions_modules+=("hudi-scanner") + extensions_modules+=("hadoop-hudi-scanner") extensions_modules+=("paimon-scanner") extensions_modules+=("trino-connector-scanner") extensions_modules+=("max-compute-scanner") diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp index 507acb00dff0b0..746f109ac6d7fd 100644 --- a/cloud/src/common/bvars.cpp +++ b/cloud/src/common/bvars.cpp @@ -198,3 +198,10 @@ BvarStatusWithTag g_bvar_checker_instance_volume("checker", "instance_volu BvarStatusWithTag g_bvar_inverted_checker_num_scanned("checker", "num_inverted_scanned"); BvarStatusWithTag g_bvar_inverted_checker_num_check_failed("checker", "num_inverted_check_failed"); + +BvarStatusWithTag g_bvar_inverted_checker_leaked_delete_bitmaps("checker", + "leaked_delete_bitmaps"); +BvarStatusWithTag g_bvar_inverted_checker_abnormal_delete_bitmaps( + "checker", "abnormal_delete_bitmaps"); +BvarStatusWithTag g_bvar_inverted_checker_delete_bitmaps_scanned( + "checker", "delete_bitmap_keys_scanned"); \ No newline at end of file diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h index 373a3a63ff2703..d0ad2e97957ae6 100644 --- a/cloud/src/common/bvars.h +++ b/cloud/src/common/bvars.h @@ -250,3 +250,7 @@ extern BvarStatusWithTag g_bvar_checker_last_success_time_ms; extern BvarStatusWithTag g_bvar_checker_instance_volume; extern BvarStatusWithTag g_bvar_inverted_checker_num_scanned; extern BvarStatusWithTag g_bvar_inverted_checker_num_check_failed; + +extern BvarStatusWithTag g_bvar_inverted_checker_leaked_delete_bitmaps; +extern BvarStatusWithTag g_bvar_inverted_checker_abnormal_delete_bitmaps; +extern BvarStatusWithTag g_bvar_inverted_checker_delete_bitmaps_scanned; diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index f6e0073b92464f..c6b6e1ef290027 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -72,6 +72,11 @@ CONF_Bool(enable_checker, "false"); CONF_Int32(recycle_pool_parallelism, "40"); // Currently only used for recycler test CONF_Bool(enable_inverted_check, "false"); +// Currently only used for recycler test +CONF_Bool(enable_delete_bitmap_inverted_check, "false"); +// checks if https://github.com/apache/doris/pull/40204 works as expected +CONF_Bool(enable_delete_bitmap_storage_optimize_check, "false"); +CONF_mInt64(delete_bitmap_storage_optimize_check_version_gap, "1000"); // interval for scanning instances to do checks and inspections CONF_mInt32(scan_instances_interval_seconds, "60"); // 1min // interval for check object diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index 87afc8ed7cef9a..18a1eefce8f002 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -1033,14 +1033,18 @@ void MetaServiceImpl::prepare_rowset(::google::protobuf::RpcController* controll prepare_rowset.SerializeToString(&val); DCHECK_GT(prepare_rowset.expiration(), 0); txn->put(prepare_rs_key, val); + std::size_t segment_key_bounds_bytes = get_segments_key_bounds_bytes(rowset_meta); LOG(INFO) << "put prepare_rs_key " << hex(prepare_rs_key) << " value_size " << val.size() - << " txn_id " << request->txn_id(); + << " txn_id " << request->txn_id() << " segment_key_bounds_bytes " + << segment_key_bounds_bytes; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { if (err == TxnErrorCode::TXN_VALUE_TOO_LARGE) { LOG(WARNING) << "failed to prepare rowset, err=value too large" << ", txn_id=" << request->txn_id() << ", tablet_id=" << tablet_id << ", rowset_id=" << rowset_id + << ", rowset_meta_bytes=" << rowset_meta.ByteSizeLong() + << ", segment_key_bounds_bytes=" << segment_key_bounds_bytes << ", rowset_meta=" << rowset_meta.ShortDebugString(); } code = cast_as(err); @@ -1167,15 +1171,18 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle DCHECK_GT(rowset_meta.txn_expiration(), 0); auto tmp_rs_val = rowset_meta.SerializeAsString(); txn->put(tmp_rs_key, tmp_rs_val); + std::size_t segment_key_bounds_bytes = get_segments_key_bounds_bytes(rowset_meta); LOG(INFO) << "put tmp_rs_key " << hex(tmp_rs_key) << " delete recycle_rs_key " << hex(recycle_rs_key) << " value_size " << tmp_rs_val.size() << " txn_id " - << request->txn_id(); + << request->txn_id() << " segment_key_bounds_bytes " << segment_key_bounds_bytes; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { if (err == TxnErrorCode::TXN_VALUE_TOO_LARGE) { LOG(WARNING) << "failed to commit rowset, err=value too large" << ", txn_id=" << request->txn_id() << ", tablet_id=" << tablet_id << ", rowset_id=" << rowset_id + << ", rowset_meta_bytes=" << rowset_meta.ByteSizeLong() + << ", segment_key_bounds_bytes=" << segment_key_bounds_bytes << ", rowset_meta=" << rowset_meta.ShortDebugString(); } code = cast_as(err); @@ -1267,10 +1274,21 @@ void MetaServiceImpl::update_tmp_rowset(::google::protobuf::RpcController* contr } txn->put(update_key, update_val); + std::size_t segment_key_bounds_bytes = get_segments_key_bounds_bytes(rowset_meta); LOG(INFO) << "xxx put " - << "update_rowset_key " << hex(update_key) << " value_size " << update_val.size(); + << "update_rowset_key " << hex(update_key) << " value_size " << update_val.size() + << " segment_key_bounds_bytes " << segment_key_bounds_bytes; err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_VALUE_TOO_LARGE) { + const auto& rowset_id = rowset_meta.rowset_id_v2(); + LOG(WARNING) << "failed to update tmp rowset, err=value too large" + << ", txn_id=" << request->txn_id() << ", tablet_id=" << tablet_id + << ", rowset_id=" << rowset_id + << ", rowset_meta_bytes=" << rowset_meta.ByteSizeLong() + << ", segment_key_bounds_bytes=" << segment_key_bounds_bytes + << ", rowset_meta=" << rowset_meta.ShortDebugString(); + } code = cast_as(err); ss << "failed to update rowset meta, err=" << err; msg = ss.str(); @@ -1310,15 +1328,27 @@ void internal_get_rowset(Transaction* txn, int64_t start, int64_t end, while (it->has_next()) { auto [k, v] = it->next(); - auto rs = response->add_rowset_meta(); + auto* rs = response->add_rowset_meta(); + auto byte_size = rs->ByteSizeLong(); + TEST_SYNC_POINT_CALLBACK("get_rowset:meta_exceed_limit", &byte_size); + if (byte_size + v.size() > std::numeric_limits::max()) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format( + "rowset meta exceeded 2G, unable to serialize, key={}. byte_size={}", + hex(k), byte_size); + LOG(WARNING) << msg; + return; + } if (!rs->ParseFromArray(v.data(), v.size())) { code = MetaServiceCode::PROTOBUF_PARSE_ERR; - msg = "malformed rowset meta, unable to deserialize"; + msg = "malformed rowset meta, unable to serialize"; LOG(WARNING) << msg << " key=" << hex(k); return; } ++num_rowsets; - if (!it->has_next()) key0 = k; + if (!it->has_next()) { + key0 = k; + } } key0.push_back('\x00'); // Update to next smallest key for iteration } while (it->more()); @@ -1750,7 +1780,7 @@ void MetaServiceImpl::update_delete_bitmap(google::protobuf::RpcController* cont } RPC_RATE_LIMIT(update_delete_bitmap) - uint32_t fdb_txn_size = 0; + uint64_t fdb_txn_size = 0; auto table_id = request->table_id(); auto tablet_id = request->tablet_id(); @@ -2372,4 +2402,12 @@ MetaServiceResponseStatus MetaServiceImpl::fix_tablet_stats(std::string cloud_un return st; } +std::size_t get_segments_key_bounds_bytes(const doris::RowsetMetaCloudPB& rowset_meta) { + size_t ret {0}; + for (const auto& key_bounds : rowset_meta.segments_key_bounds()) { + ret += key_bounds.ByteSizeLong(); + } + return ret; +} + } // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 7af96cbc14b8ee..5374cbea741fb0 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -729,6 +729,7 @@ class MetaServiceProxy final : public MetaService { int32_t retry_times = 0; uint64_t duration_ms = 0, retry_drift_ms = 0; while (true) { + resp->Clear(); // reset the response message in case it is reused for retry (impl_.get()->*method)(ctrl, req, resp, brpc::DoNothing()); MetaServiceCode code = resp->status().code(); if (code != MetaServiceCode::KV_TXN_STORE_GET_RETRYABLE && diff --git a/cloud/src/meta-service/meta_service_helper.h b/cloud/src/meta-service/meta_service_helper.h index 4ef98ea2fb7b0c..e4ed7e2a231901 100644 --- a/cloud/src/meta-service/meta_service_helper.h +++ b/cloud/src/meta-service/meta_service_helper.h @@ -118,6 +118,12 @@ void finish_rpc(std::string_view func_name, brpc::Controller* ctrl, Response* re << " status=" << res->status().ShortDebugString() << " tablet=" << res->tablet_id() << " delete_bitmap_count=" << res->segment_delete_bitmaps_size(); + } else if constexpr (std::is_same_v) { + if (res->status().code() != MetaServiceCode::OK) { + res->clear_base_compaction_cnts(); + res->clear_cumulative_compaction_cnts(); + res->clear_cumulative_points(); + } } else if constexpr (std::is_same_v || std::is_same_v) { std::string debug_string = res->DebugString(); @@ -248,4 +254,5 @@ void get_tablet_idx(MetaServiceCode& code, std::string& msg, Transaction* txn, bool is_dropped_tablet(Transaction* txn, const std::string& instance_id, int64_t index_id, int64_t partition_id); +std::size_t get_segments_key_bounds_bytes(const doris::RowsetMetaCloudPB& rowset_meta); } // namespace doris::cloud diff --git a/cloud/src/recycler/checker.cpp b/cloud/src/recycler/checker.cpp index 19a10d61c12d35..60b6b7fc5eefea 100644 --- a/cloud/src/recycler/checker.cpp +++ b/cloud/src/recycler/checker.cpp @@ -168,17 +168,35 @@ int Checker::start() { auto ctime_ms = duration_cast(system_clock::now().time_since_epoch()).count(); g_bvar_checker_enqueue_cost_s.put(instance_id, ctime_ms / 1000 - enqueue_time_s); - int ret1 = checker->do_check(); - int ret2 = 0; + bool success {true}; + + if (int ret = checker->do_check(); ret != 0) { + success = false; + } + if (config::enable_inverted_check) { - ret2 = checker->do_inverted_check(); + if (int ret = checker->do_inverted_check(); ret != 0) { + success = false; + } + } + + if (config::enable_delete_bitmap_inverted_check) { + if (int ret = checker->do_delete_bitmap_inverted_check(); ret != 0) { + success = false; + } + } + + if (config::enable_delete_bitmap_storage_optimize_check) { + if (int ret = checker->do_delete_bitmap_storage_optimize_check(); ret != 0) { + success = false; + } } // If instance checker has been aborted, don't finish this job if (!checker->stopped()) { finish_instance_recycle_job(txn_kv_.get(), check_job_key, instance.instance_id(), - ip_port_, ret1 == 0 && ret2 == 0, ctime_ms); + ip_port_, success, ctime_ms); } { std::lock_guard lock(mtx_); @@ -740,4 +758,435 @@ int InstanceChecker::do_inverted_check() { return num_file_leak > 0 ? 1 : check_ret; } +int InstanceChecker::traverse_mow_tablet(const std::function& check_func) { + std::unique_ptr it; + auto begin = meta_rowset_key({instance_id_, 0, 0}); + auto end = meta_rowset_key({instance_id_, std::numeric_limits::max(), 0}); + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + err = txn->get(begin, end, &it, false, 1); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get rowset kv, err=" << err; + return -1; + } + if (!it->has_next()) { + break; + } + while (it->has_next() && !stopped()) { + auto [k, v] = it->next(); + std::string_view k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "meta" ${instance_id} "rowset" ${tablet_id} ${version} -> RowsetMetaCloudPB + auto tablet_id = std::get(std::get<0>(out[3])); + + if (!it->has_next()) { + // Update to next smallest key for iteration + // scan for next tablet in this instance + begin = meta_rowset_key({instance_id_, tablet_id + 1, 0}); + } + + TabletMetaCloudPB tablet_meta; + int ret = get_tablet_meta(txn_kv_.get(), instance_id_, tablet_id, tablet_meta); + if (ret < 0) { + LOG(WARNING) << fmt::format( + "failed to get_tablet_meta in do_delete_bitmap_integrity_check(), " + "instance_id={}, tablet_id={}", + instance_id_, tablet_id); + return ret; + } + + if (tablet_meta.enable_unique_key_merge_on_write()) { + // only check merge-on-write table + int ret = check_func(tablet_id); + if (ret < 0) { + // return immediately when encounter unexpected error, + // otherwise, we continue to check the next tablet + return ret; + } + } + } + } while (it->more() && !stopped()); + return 0; +} + +int InstanceChecker::traverse_rowset_delete_bitmaps( + int64_t tablet_id, std::string rowset_id, + const std::function& callback) { + std::unique_ptr it; + auto begin = meta_delete_bitmap_key({instance_id_, tablet_id, rowset_id, 0, 0}); + auto end = meta_delete_bitmap_key({instance_id_, tablet_id, rowset_id, + std::numeric_limits::max(), + std::numeric_limits::max()}); + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + err = txn->get(begin, end, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get rowset kv, err=" << err; + return -1; + } + if (!it->has_next()) { + break; + } + while (it->has_next() && !stopped()) { + auto [k, v] = it->next(); + std::string_view k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "meta" ${instance_id} "delete_bitmap" ${tablet_id} ${rowset_id} ${version} ${segment_id} -> roaringbitmap + auto version = std::get(std::get<0>(out[5])); + auto segment_id = std::get(std::get<0>(out[6])); + + int ret = callback(tablet_id, rowset_id, version, segment_id); + if (ret != 0) { + return ret; + } + + if (!it->has_next()) { + begin = k; + begin.push_back('\x00'); // Update to next smallest key for iteration + break; + } + } + } while (it->more() && !stopped()); + + return 0; +} + +int InstanceChecker::collect_tablet_rowsets( + int64_t tablet_id, const std::function& collect_cb) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + std::unique_ptr it; + auto begin = meta_rowset_key({instance_id_, tablet_id, 0}); + auto end = meta_rowset_key({instance_id_, tablet_id + 1, 0}); + + int64_t rowsets_num {0}; + do { + TxnErrorCode err = txn->get(begin, end, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get rowset kv, err=" << err; + return -1; + } + if (!it->has_next()) { + break; + } + while (it->has_next() && !stopped()) { + auto [k, v] = it->next(); + doris::RowsetMetaCloudPB rowset; + if (!rowset.ParseFromArray(v.data(), v.size())) { + LOG(WARNING) << "malformed rowset meta value, key=" << hex(k); + return -1; + } + + ++rowsets_num; + collect_cb(rowset); + + if (!it->has_next()) { + begin = k; + begin.push_back('\x00'); // Update to next smallest key for iteration + break; + } + } + } while (it->more() && !stopped()); + + LOG(INFO) << fmt::format( + "[delete bitmap checker] successfully collect rowsets for instance_id={}, " + "tablet_id={}, rowsets_num={}", + instance_id_, tablet_id, rowsets_num); + return 0; +} + +int InstanceChecker::do_delete_bitmap_inverted_check() { + LOG(INFO) << fmt::format( + "[delete bitmap checker] begin to do_delete_bitmap_inverted_check for instance_id={}", + instance_id_); + + // number of delete bitmap keys being scanned + int64_t total_delete_bitmap_keys {0}; + // number of delete bitmaps which belongs to non mow tablet + int64_t abnormal_delete_bitmaps {0}; + // number of delete bitmaps which doesn't have corresponding rowset in MS + int64_t leaked_delete_bitmaps {0}; + + auto start_time = std::chrono::steady_clock::now(); + std::unique_ptr> defer_log_statistics((int*)0x01, [&](int*) { + g_bvar_inverted_checker_leaked_delete_bitmaps.put(instance_id_, leaked_delete_bitmaps); + g_bvar_inverted_checker_abnormal_delete_bitmaps.put(instance_id_, abnormal_delete_bitmaps); + g_bvar_inverted_checker_delete_bitmaps_scanned.put(instance_id_, total_delete_bitmap_keys); + + auto cost = std::chrono::duration_cast( + std::chrono::steady_clock::now() - start_time) + .count(); + if (leaked_delete_bitmaps > 0 || abnormal_delete_bitmaps > 0) { + LOG(WARNING) << fmt::format( + "[delete bitmap check fails] delete bitmap inverted check for instance_id={}, " + "cost={} ms, total_delete_bitmap_keys={}, leaked_delete_bitmaps={}, " + "abnormal_delete_bitmaps={}", + instance_id_, cost, total_delete_bitmap_keys, leaked_delete_bitmaps, + abnormal_delete_bitmaps); + } else { + LOG(INFO) << fmt::format( + "[delete bitmap checker] delete bitmap inverted check for instance_id={}, " + "passed. cost={} ms, total_delete_bitmap_keys={}", + instance_id_, cost, total_delete_bitmap_keys); + } + }); + + struct TabletsRowsetsCache { + int64_t tablet_id {-1}; + bool enable_merge_on_write {false}; + std::unordered_set rowsets {}; + } tablet_rowsets_cache {}; + + std::unique_ptr it; + auto begin = meta_delete_bitmap_key({instance_id_, 0, "", 0, 0}); + auto end = + meta_delete_bitmap_key({instance_id_, std::numeric_limits::max(), "", 0, 0}); + do { + std::unique_ptr txn; + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + err = txn->get(begin, end, &it); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to get rowset kv, err=" << err; + return -1; + } + if (!it->has_next()) { + break; + } + while (it->has_next() && !stopped()) { + auto [k, v] = it->next(); + std::string_view k1 = k; + k1.remove_prefix(1); + std::vector, int, int>> out; + decode_key(&k1, &out); + // 0x01 "meta" ${instance_id} "delete_bitmap" ${tablet_id} ${rowset_id} ${version} ${segment_id} -> roaringbitmap + auto tablet_id = std::get(std::get<0>(out[3])); + auto rowset_id = std::get(std::get<0>(out[4])); + auto version = std::get(std::get<0>(out[5])); + auto segment_id = std::get(std::get<0>(out[6])); + + ++total_delete_bitmap_keys; + + if (!it->has_next()) { + begin = k; + begin.push_back('\x00'); // Update to next smallest key for iteration + } + + if (tablet_rowsets_cache.tablet_id == -1 || + tablet_rowsets_cache.tablet_id != tablet_id) { + TabletMetaCloudPB tablet_meta; + int ret = get_tablet_meta(txn_kv_.get(), instance_id_, tablet_id, tablet_meta); + if (ret < 0) { + LOG(WARNING) << fmt::format( + "[delete bitmap checker] failed to get_tablet_meta in " + "do_delete_bitmap_inverted_check(), instance_id={}, tablet_id={}", + instance_id_, tablet_id); + return ret; + } + + tablet_rowsets_cache.tablet_id = tablet_id; + tablet_rowsets_cache.enable_merge_on_write = + tablet_meta.enable_unique_key_merge_on_write(); + tablet_rowsets_cache.rowsets.clear(); + + if (tablet_rowsets_cache.enable_merge_on_write) { + // only collect rowsets for merge-on-write tablet + auto collect_cb = + [&tablet_rowsets_cache](const doris::RowsetMetaCloudPB& rowset) { + tablet_rowsets_cache.rowsets.insert(rowset.rowset_id_v2()); + }; + ret = collect_tablet_rowsets(tablet_id, collect_cb); + if (ret < 0) { + return ret; + } + } + } + DCHECK_EQ(tablet_id, tablet_rowsets_cache.tablet_id); + + if (!tablet_rowsets_cache.enable_merge_on_write) { + // clang-format off + TEST_SYNC_POINT_CALLBACK( + "InstanceChecker::do_delete_bitmap_inverted_check.get_abnormal_delete_bitmap", + &tablet_id, &rowset_id, &version, &segment_id); + // clang-format on + ++abnormal_delete_bitmaps; + // log an error and continue to check the next delete bitmap + LOG(WARNING) << fmt::format( + "[delete bitmap check fails] find a delete bitmap belongs to tablet " + "which is not a merge-on-write table! instance_id={}, tablet_id={}, " + "version={}, segment_id={}", + instance_id_, tablet_id, version, segment_id); + continue; + } + + if (!tablet_rowsets_cache.rowsets.contains(rowset_id)) { + TEST_SYNC_POINT_CALLBACK( + "InstanceChecker::do_delete_bitmap_inverted_check.get_leaked_delete_bitmap", + &tablet_id, &rowset_id, &version, &segment_id); + ++leaked_delete_bitmaps; + // log an error and continue to check the next delete bitmap + LOG(WARNING) << fmt::format( + "[delete bitmap check fails] can't find corresponding rowset for delete " + "bitmap instance_id={}, tablet_id={}, rowset_id={}, version={}, " + "segment_id={}", + instance_id_, tablet_id, rowset_id, version, segment_id); + } + } + } while (it->more() && !stopped()); + + return (leaked_delete_bitmaps > 0 || abnormal_delete_bitmaps > 0) ? 1 : 0; +} + +int InstanceChecker::check_delete_bitmap_storage_optimize(int64_t tablet_id) { + using Version = std::pair; + struct RowsetDigest { + std::string rowset_id; + Version version; + doris::SegmentsOverlapPB segments_overlap; + + bool operator<(const RowsetDigest& other) const { + return version.first < other.version.first; + } + + bool produced_by_compaction() const { + return (version.first < version.second) || + ((version.first == version.second) && segments_overlap == NONOVERLAPPING); + } + }; + + // number of rowsets which may have problems + int64_t abnormal_rowsets_num {0}; + + std::vector tablet_rowsets {}; + // Get all visible rowsets of this tablet + auto collect_cb = [&tablet_rowsets](const doris::RowsetMetaCloudPB& rowset) { + if (rowset.start_version() == 0 && rowset.end_version() == 1) { + // ignore dummy rowset [0-1] + return; + } + tablet_rowsets.emplace_back( + rowset.rowset_id_v2(), + std::make_pair(rowset.start_version(), rowset.end_version()), + rowset.segments_overlap_pb()); + }; + if (int ret = collect_tablet_rowsets(tablet_id, collect_cb); ret != 0) { + return ret; + } + + std::sort(tablet_rowsets.begin(), tablet_rowsets.end()); + + // find right-most rowset which is produced by compaction + auto it = std::find_if( + tablet_rowsets.crbegin(), tablet_rowsets.crend(), + [](const RowsetDigest& rowset) { return rowset.produced_by_compaction(); }); + if (it == tablet_rowsets.crend()) { + LOG(INFO) << fmt::format( + "[delete bitmap checker] skip to check delete bitmap storage optimize for " + "tablet_id={} because it doesn't have compacted rowsets.", + tablet_id); + return 0; + } + + int64_t start_version = it->version.first; + int64_t pre_min_version = it->version.second; + + // after BE sweeping stale rowsets, all rowsets in this tablet before + // should not have delete bitmaps with versions lower than `pre_min_version` + if (config::delete_bitmap_storage_optimize_check_version_gap > 0) { + pre_min_version -= config::delete_bitmap_storage_optimize_check_version_gap; + if (pre_min_version <= 1) { + LOG(INFO) << fmt::format( + "[delete bitmap checker] skip to check delete bitmap storage optimize for " + "tablet_id={} because pre_min_version is too small.", + tablet_id); + return 0; + } + } + + auto check_func = [pre_min_version, instance_id = instance_id_]( + int64_t tablet_id, std::string_view rowset_id, int64_t version, + int64_t segment_id) -> int { + if (version < pre_min_version) { + LOG(WARNING) << fmt::format( + "[delete bitmap check fails] delete bitmap storage optimize check fail for " + "instance_id={}, tablet_id={}, rowset_id={}, found delete bitmap with " + "version={} < pre_min_version={}", + instance_id, tablet_id, rowset_id, version, pre_min_version); + return 1; + } + return 0; + }; + + for (const auto& rowset : tablet_rowsets) { + // check for all rowsets before the max compacted rowset + if (rowset.version.second < start_version) { + auto rowset_id = rowset.rowset_id; + int ret = traverse_rowset_delete_bitmaps(tablet_id, rowset_id, check_func); + if (ret < 0) { + return ret; + } + + if (ret != 0) { + ++abnormal_rowsets_num; + TEST_SYNC_POINT_CALLBACK( + "InstanceChecker::check_delete_bitmap_storage_optimize.get_abnormal_rowset", + &tablet_id, &rowset_id); + } + } + } + + LOG(INFO) << fmt::format( + "[delete bitmap checker] finish check delete bitmap storage optimize for " + "instance_id={}, tablet_id={}, rowsets_num={}, abnormal_rowsets_num={}, " + "pre_min_version={}", + instance_id_, tablet_id, tablet_rowsets.size(), abnormal_rowsets_num, pre_min_version); + + return (abnormal_rowsets_num > 1 ? 1 : 0); +} + +int InstanceChecker::do_delete_bitmap_storage_optimize_check() { + int64_t total_tablets_num {0}; + int64_t failed_tablets_num {0}; + + // check that for every visible rowset, there exists at least delete one bitmap in MS + int ret = traverse_mow_tablet([&](int64_t tablet_id) { + ++total_tablets_num; + int res = check_delete_bitmap_storage_optimize(tablet_id); + failed_tablets_num += (res != 0); + return res; + }); + + if (ret < 0) { + return ret; + } + + LOG(INFO) << fmt::format( + "[delete bitmap checker] check delete bitmap storage optimize for instance_id={}, " + "total_tablets_num={}, failed_tablets_num={}", + instance_id_, total_tablets_num, failed_tablets_num); + + return (failed_tablets_num > 0) ? 1 : 0; +} + } // namespace doris::cloud diff --git a/cloud/src/recycler/checker.h b/cloud/src/recycler/checker.h index 03717a69b5ea20..7f87e90f7cb366 100644 --- a/cloud/src/recycler/checker.h +++ b/cloud/src/recycler/checker.h @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -30,6 +31,10 @@ #include "recycler/storage_vault_accessor.h" #include "recycler/white_black_list.h" +namespace doris { +class RowsetMetaCloudPB; +} // namespace doris + namespace doris::cloud { class StorageVaultAccessor; class InstanceChecker; @@ -86,6 +91,18 @@ class InstanceChecker { // Return 1 if data loss is identified. // Return negative if a temporary error occurred during the check process. int do_check(); + + // Return 0 if success. + // Return 1 if delete bitmap leak is identified. + // Return negative if a temporary error occurred during the check process. + int do_delete_bitmap_inverted_check(); + + // checks if https://github.com/apache/doris/pull/40204 works as expected + // the stale delete bitmap will be cleared in MS when BE delete expired stale rowsets + // NOTE: stale rowsets will be lost after BE restarts, so there may be some stale delete bitmaps + // which will not be cleared. + int do_delete_bitmap_storage_optimize_check(); + // If there are multiple buckets, return the minimum lifecycle; if there are no buckets (i.e. // all accessors are HdfsAccessor), return INT64_MAX. // Return 0 if success, otherwise error @@ -100,6 +117,17 @@ class InstanceChecker { // returns 0 for success otherwise error int init_storage_vault_accessors(const InstanceInfoPB& instance); + int traverse_mow_tablet(const std::function& check_func); + int traverse_rowset_delete_bitmaps( + int64_t tablet_id, std::string rowset_id, + const std::function& callback); + int collect_tablet_rowsets( + int64_t tablet_id, + const std::function& collect_cb); + int traverse_delete_bitmaps(const std::function& check_func); + + int check_delete_bitmap_storage_optimize(int64_t tablet_id); + std::atomic_bool stopped_ {false}; std::shared_ptr txn_kv_; std::string instance_id_; diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index 1b21ec68916be6..f7000ea379225f 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -417,8 +417,12 @@ class InstanceRecycler::InvertedIndexIdCache { LOG(WARNING) << "malformed schema value, key=" << hex(schema_key); return -1; } - if (schema.index_size() > 0 && schema.has_inverted_index_storage_format()) { - res.first = schema.inverted_index_storage_format(); + if (schema.index_size() > 0) { + InvertedIndexStorageFormatPB index_format = InvertedIndexStorageFormatPB::V1; + if (schema.has_inverted_index_storage_format()) { + index_format = schema.inverted_index_storage_format(); + } + res.first = index_format; res.second.reserve(schema.index_size()); for (auto& i : schema.index()) { if (i.has_index_type() && i.index_type() == IndexType::INVERTED) { @@ -1382,17 +1386,19 @@ int InstanceRecycler::delete_rowset_data(const doris::RowsetMetaCloudPB& rs_meta } std::vector file_paths; auto tablet_schema = rs_meta_pb.tablet_schema(); + auto index_storage_format = InvertedIndexStorageFormatPB::V1; for (int64_t i = 0; i < num_segments; ++i) { file_paths.push_back(segment_path(tablet_id, rowset_id, i)); if (tablet_schema.has_inverted_index_storage_format()) { - if (tablet_schema.inverted_index_storage_format() == InvertedIndexStorageFormatPB::V1) { - for (const auto& index_id : index_ids) { - file_paths.push_back(inverted_index_path_v1(tablet_id, rowset_id, i, - index_id.first, index_id.second)); - } - } else if (!index_ids.empty()) { - file_paths.push_back(inverted_index_path_v2(tablet_id, rowset_id, i)); + index_storage_format = tablet_schema.inverted_index_storage_format(); + } + if (index_storage_format == InvertedIndexStorageFormatPB::V1) { + for (const auto& index_id : index_ids) { + file_paths.push_back(inverted_index_path_v1(tablet_id, rowset_id, i, index_id.first, + index_id.second)); } + } else if (!index_ids.empty()) { + file_paths.push_back(inverted_index_path_v2(tablet_id, rowset_id, i)); } } // TODO(AlexYue): seems could do do batch @@ -1429,8 +1435,8 @@ int InstanceRecycler::delete_rowset_data(const std::vector> index_ids; - // default format as v2. - InvertedIndexStorageFormatPB index_format = InvertedIndexStorageFormatPB::V2; + // default format as v1. + InvertedIndexStorageFormatPB index_format = InvertedIndexStorageFormatPB::V1; if (rs.has_tablet_schema()) { for (const auto& index : rs.tablet_schema().index()) { diff --git a/cloud/src/recycler/s3_accessor.cpp b/cloud/src/recycler/s3_accessor.cpp index 2c983a5fa0617b..1aca88d2d1161d 100644 --- a/cloud/src/recycler/s3_accessor.cpp +++ b/cloud/src/recycler/s3_accessor.cpp @@ -205,6 +205,7 @@ std::optional S3Conf::from_obj_store_info(const ObjectStoreInfoPB& obj_i s3_conf.region = obj_info.region(); s3_conf.bucket = obj_info.bucket(); s3_conf.prefix = obj_info.prefix(); + s3_conf.use_virtual_addressing = !obj_info.use_path_style(); return s3_conf; } @@ -289,7 +290,7 @@ int S3Accessor::init() { auto s3_client = std::make_shared( std::move(aws_cred), std::move(aws_config), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - true /* useVirtualAddressing */); + conf_.use_virtual_addressing /* useVirtualAddressing */); obj_client_ = std::make_shared(std::move(s3_client), conf_.endpoint); return 0; } diff --git a/cloud/src/recycler/s3_accessor.h b/cloud/src/recycler/s3_accessor.h index 6886ee5e7c5640..e9640b5693a1f5 100644 --- a/cloud/src/recycler/s3_accessor.h +++ b/cloud/src/recycler/s3_accessor.h @@ -69,6 +69,7 @@ struct S3Conf { std::string region; std::string bucket; std::string prefix; + bool use_virtual_addressing {true}; enum Provider : uint8_t { S3, diff --git a/cloud/src/recycler/s3_obj_client.cpp b/cloud/src/recycler/s3_obj_client.cpp index 53fa821c7e5503..c8dcdad18d7115 100644 --- a/cloud/src/recycler/s3_obj_client.cpp +++ b/cloud/src/recycler/s3_obj_client.cpp @@ -284,6 +284,7 @@ ObjectStorageResponse S3ObjClient::delete_object(ObjectStoragePathRef path) { SCOPED_BVAR_LATENCY(s3_bvar::s3_delete_object_latency); return s3_client_->DeleteObject(request); }); + TEST_SYNC_POINT_CALLBACK("S3ObjClient::delete_object", &outcome); if (!outcome.IsSuccess()) { LOG_WARNING("failed to delete object") .tag("endpoint", endpoint_) diff --git a/cloud/src/recycler/util.cpp b/cloud/src/recycler/util.cpp index d6c63ea752f4f1..6797782d5d665c 100644 --- a/cloud/src/recycler/util.cpp +++ b/cloud/src/recycler/util.cpp @@ -23,6 +23,7 @@ #include "common/util.h" #include "meta-service/keys.h" +#include "meta-service/meta_service_schema.h" #include "meta-service/txn_kv.h" #include "meta-service/txn_kv_error.h" @@ -233,4 +234,69 @@ int lease_instance_recycle_job(TxnKv* txn_kv, std::string_view key, const std::s return 0; } +int get_tablet_idx(TxnKv* txn_kv, const std::string& instance_id, int64_t tablet_id, + TabletIndexPB& tablet_idx) { + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + + std::string key, val; + meta_tablet_idx_key({instance_id, tablet_id}, &key); + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << fmt::format("failed to get tablet_idx, err={} tablet_id={} key={}", err, + tablet_id, hex(key)); + return -1; + } + if (!tablet_idx.ParseFromString(val)) [[unlikely]] { + LOG(WARNING) << fmt::format("malformed tablet index value, tablet_id={} key={}", tablet_id, + hex(key)); + return -1; + } + if (tablet_id != tablet_idx.tablet_id()) [[unlikely]] { + LOG(WARNING) << "unexpected error given_tablet_id=" << tablet_id + << " idx_pb_tablet_id=" << tablet_idx.tablet_id() << " key=" << hex(key); + return -1; + } + return 0; +} + +int get_tablet_meta(TxnKv* txn_kv, const std::string& instance_id, int64_t tablet_id, + TabletMetaCloudPB& tablet_meta) { + TabletIndexPB tablet_idx; + int ret = get_tablet_idx(txn_kv, instance_id, tablet_id, tablet_idx); + if (ret < 0) { + return ret; + } + + std::unique_ptr txn; + TxnErrorCode err = txn_kv->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << "failed to create txn"; + return -1; + } + + std::string key, val; + meta_tablet_key({instance_id, tablet_idx.table_id(), tablet_idx.index_id(), + tablet_idx.partition_id(), tablet_id}, + &key); + err = txn->get(key, &val); + if (err != TxnErrorCode::TXN_OK) { + LOG(WARNING) << fmt::format( + "failed to get tablet, err={}, table_id={}, index_id={}, partition_id={}, " + "tablet_id={} key={}", + err, tablet_idx.table_id(), tablet_idx.index_id(), tablet_idx.partition_id(), + tablet_id, hex(key)); + return -1; + } + if (!tablet_meta.ParseFromString(val)) [[unlikely]] { + LOG(WARNING) << fmt::format("malformed tablet meta, tablet_id={} key={}", tablet_id, + hex(key)); + return -1; + } + return 0; +} } // namespace doris::cloud diff --git a/cloud/src/recycler/util.h b/cloud/src/recycler/util.h index b6d4d3299b5e0f..5aa929c23984b1 100644 --- a/cloud/src/recycler/util.h +++ b/cloud/src/recycler/util.h @@ -85,4 +85,9 @@ inline std::string tablet_path_prefix(int64_t tablet_id) { return fmt::format("data/{}/", tablet_id); } +int get_tablet_idx(TxnKv* txn_kv, const std::string& instance_id, int64_t tablet_id, + TabletIndexPB& tablet_idx); + +int get_tablet_meta(TxnKv* txn_kv, const std::string& instance_id, int64_t tablet_id, + TabletMetaCloudPB& tablet_meta); } // namespace doris::cloud diff --git a/cloud/test/recycler_test.cpp b/cloud/test/recycler_test.cpp index 146873548398bf..0bc16644a82041 100644 --- a/cloud/test/recycler_test.cpp +++ b/cloud/test/recycler_test.cpp @@ -255,14 +255,63 @@ static int create_committed_rowset(TxnKv* txn_kv, StorageVaultAccessor* accessor return 0; } +static int create_committed_rowset_with_rowset_id(TxnKv* txn_kv, StorageVaultAccessor* accessor, + const std::string& resource_id, int64_t tablet_id, + int64_t start_version, int64_t end_version, + std::string rowset_id, bool segments_overlap, + int num_segments) { + std::string key; + std::string val; + + MetaRowsetKeyInfo key_info {instance_id, tablet_id, end_version}; + meta_rowset_key(key_info, &key); + + doris::RowsetMetaCloudPB rowset_pb; + rowset_pb.set_rowset_id(0); // useless but required + rowset_pb.set_rowset_id_v2(rowset_id); + rowset_pb.set_num_segments(num_segments); + rowset_pb.set_tablet_id(tablet_id); + rowset_pb.set_resource_id(resource_id); + rowset_pb.set_creation_time(current_time); + rowset_pb.set_start_version(start_version); + rowset_pb.set_end_version(end_version); + rowset_pb.set_segments_overlap_pb(segments_overlap ? OVERLAPPING : NONOVERLAPPING); + rowset_pb.SerializeToString(&val); + + std::unique_ptr txn; + if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { + return -1; + } + txn->put(key, val); + if (txn->commit() != TxnErrorCode::TXN_OK) { + return -1; + } + + for (int i = 0; i < num_segments; ++i) { + auto path = segment_path(tablet_id, rowset_id, i); + accessor->put_file(path, ""); + } + return 0; +} + +static void create_delete_bitmaps(Transaction* txn, int64_t tablet_id, std::string rowset_id, + int64_t start_version, int64_t end_version) { + for (int64_t ver {start_version}; ver <= end_version; ver++) { + auto key = meta_delete_bitmap_key({instance_id, tablet_id, rowset_id, ver, 0}); + std::string val {"test_data"}; + txn->put(key, val); + } +} + static int create_tablet(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int64_t partition_id, - int64_t tablet_id) { + int64_t tablet_id, bool is_mow = false) { std::unique_ptr txn; if (txn_kv->create_txn(&txn) != TxnErrorCode::TXN_OK) { return -1; } doris::TabletMetaCloudPB tablet_meta; tablet_meta.set_tablet_id(tablet_id); + tablet_meta.set_enable_unique_key_merge_on_write(is_mow); auto val = tablet_meta.SerializeAsString(); auto key = meta_tablet_key({instance_id, table_id, index_id, partition_id, tablet_id}); txn->put(key, val); @@ -275,6 +324,7 @@ static int create_tablet(TxnKv* txn_kv, int64_t table_id, int64_t index_id, int6 TabletIndexPB tablet_idx_pb; tablet_idx_pb.set_db_id(db_id); tablet_idx_pb.set_table_id(table_id); + tablet_idx_pb.set_index_id(index_id); tablet_idx_pb.set_partition_id(partition_id); tablet_idx_pb.set_tablet_id(tablet_id); auto idx_val = tablet_idx_pb.SerializeAsString(); @@ -2576,6 +2626,352 @@ TEST(CheckerTest, do_inspect) { } } +TEST(CheckerTest, delete_bitmap_inverted_check_normal) { + // normal case, all delete bitmaps belong to a rowset + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + auto accessor = checker.accessor_map_.begin()->second; + + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002; + // create some rowsets with delete bitmaps in merge-on-write tablet + for (int tablet_id = 600001; tablet_id <= 600010; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + int64_t rowset_start_id = 400; + for (int ver = 2; ver <= 10; ++ver) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + ver, ver, rowset_id, false, 1); + if (ver >= 5) { + auto delete_bitmap_key = + meta_delete_bitmap_key({instance_id, tablet_id, rowset_id, ver, 0}); + std::string delete_bitmap_val {"test"}; + txn->put(delete_bitmap_key, delete_bitmap_val); + } else { + // delete bitmaps may be spilitted into mulitiple KVs if too large + auto delete_bitmap_key = + meta_delete_bitmap_key({instance_id, tablet_id, rowset_id, ver, 0}); + std::string delete_bitmap_val(1000, 'A'); + cloud::put(txn.get(), delete_bitmap_key, delete_bitmap_val, 0, 300); + } + } + } + + // also create some rowsets without delete bitmaps in non merge-on-write tablet + for (int tablet_id = 700001; tablet_id <= 700010; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, false)); + int64_t rowset_start_id = 500; + for (int ver = 2; ver < 10; ++ver) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + ver, ver, rowset_id, false, 1); + } + } + + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + + ASSERT_EQ(checker.do_delete_bitmap_inverted_check(), 0); +} + +TEST(CheckerTest, delete_bitmap_inverted_check_abnormal) { + // abnormal case, some delete bitmaps arem leaked + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + auto accessor = checker.accessor_map_.begin()->second; + + // tablet_id -> [rowset_id, version, segment_id] + std::map>> + expected_abnormal_delete_bitmaps {}, real_abnormal_delete_bitmaps {}; + std::map>> + expected_leaked_delete_bitmaps {}, real_leaked_delete_bitmaps {}; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back( + "InstanceChecker::do_delete_bitmap_inverted_check.get_abnormal_delete_bitmap", + [&real_abnormal_delete_bitmaps](auto&& args) { + int64_t tablet_id = *try_any_cast(args[0]); + std::string rowset_id = *try_any_cast(args[1]); + int64_t version = *try_any_cast(args[2]); + int64_t segment_id = *try_any_cast(args[3]); + real_abnormal_delete_bitmaps[tablet_id].insert({rowset_id, version, segment_id}); + }); + sp->set_call_back( + "InstanceChecker::do_delete_bitmap_inverted_check.get_leaked_delete_bitmap", + [&real_leaked_delete_bitmaps](auto&& args) { + int64_t tablet_id = *try_any_cast(args[0]); + std::string rowset_id = *try_any_cast(args[1]); + int64_t version = *try_any_cast(args[2]); + int64_t segment_id = *try_any_cast(args[3]); + real_leaked_delete_bitmaps[tablet_id].insert({rowset_id, version, segment_id}); + }); + sp->enable_processing(); + + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002; + // create some rowsets with delete bitmaps in merge-on-write tablet + for (int tablet_id = 800001; tablet_id <= 800010; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + int64_t rowset_start_id = 600; + for (int ver = 2; ver <= 20; ++ver) { + std::string rowset_id = std::to_string(rowset_start_id++); + + if (ver >= 10) { + // only create rowsets for some versions + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + ver, ver, rowset_id, false, 1); + } else { + expected_leaked_delete_bitmaps[tablet_id].insert({rowset_id, ver, 0}); + } + + if (ver >= 5) { + auto delete_bitmap_key = + meta_delete_bitmap_key({instance_id, tablet_id, rowset_id, ver, 0}); + std::string delete_bitmap_val {"test"}; + txn->put(delete_bitmap_key, delete_bitmap_val); + } else { + // delete bitmaps may be spilitted into mulitiple KVs if too large + auto delete_bitmap_key = + meta_delete_bitmap_key({instance_id, tablet_id, rowset_id, ver, 0}); + std::string delete_bitmap_val(1000, 'A'); + cloud::put(txn.get(), delete_bitmap_key, delete_bitmap_val, 0, 300); + } + } + } + + // create some rowsets with delete bitmaps in non merge-on-write tablet + for (int tablet_id = 900001; tablet_id <= 900010; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, false)); + int64_t rowset_start_id = 700; + for (int ver = 2; ver < 6; ++ver) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + ver, ver, rowset_id, false, 1); + auto delete_bitmap_key = + meta_delete_bitmap_key({instance_id, tablet_id, rowset_id, ver, 0}); + std::string delete_bitmap_val {"test2"}; + txn->put(delete_bitmap_key, delete_bitmap_val); + + expected_abnormal_delete_bitmaps[tablet_id].insert({rowset_id, ver, 0}); + } + } + + // create some rowsets without delete bitmaps in non merge-on-write tablet + for (int tablet_id = 700001; tablet_id <= 700010; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, false)); + int64_t rowset_start_id = 500; + for (int ver = 2; ver < 10; ++ver) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + ver, ver, rowset_id, false, 1); + } + } + + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + + ASSERT_EQ(checker.do_delete_bitmap_inverted_check(), 1); + ASSERT_EQ(expected_leaked_delete_bitmaps, real_leaked_delete_bitmaps); + ASSERT_EQ(expected_abnormal_delete_bitmaps, real_abnormal_delete_bitmaps); +} + +TEST(CheckerTest, delete_bitmap_storage_optimize_check_normal) { + config::delete_bitmap_storage_optimize_check_version_gap = 0; + + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + auto accessor = checker.accessor_map_.begin()->second; + + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002; + int64_t rowset_start_id = 600; + + for (int tablet_id = 800001; tablet_id <= 800005; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + std::vector> rowset_vers {{2, 2}, {3, 3}, {4, 4}, {5, 5}, + {6, 7}, {8, 8}, {9, 9}}; + std::vector> delete_bitmaps_vers { + {7, 9}, {8, 9}, {7, 9}, {7, 9}, {7, 9}, {8, 9}, {9, 9}}; + std::vector segments_overlap {true, true, true, true, false, true, true}; + for (size_t i {0}; i < 7; i++) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + rowset_vers[i].first, rowset_vers[i].second, + rowset_id, segments_overlap[i], 1); + create_delete_bitmaps(txn.get(), tablet_id, rowset_id, delete_bitmaps_vers[i].first, + delete_bitmaps_vers[i].second); + } + } + + for (int tablet_id = 800006; tablet_id <= 800010; ++tablet_id) { + // [7-7] cumu compaction output rowset start_version == end_version + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + std::vector> rowset_vers {{2, 2}, {3, 3}, {4, 4}, {5, 5}, + {6, 6}, {7, 7}, {8, 8}, {9, 9}}; + std::vector> delete_bitmaps_vers { + {7, 9}, {8, 9}, {7, 9}, {7, 9}, {7, 9}, {7, 9}, {8, 9}, {9, 9}}; + std::vector segments_overlap {true, true, false, true, false, true, true, true}; + for (size_t i {0}; i < 8; i++) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + rowset_vers[i].first, rowset_vers[i].second, + rowset_id, segments_overlap[i], 1); + create_delete_bitmaps(txn.get(), tablet_id, rowset_id, delete_bitmaps_vers[i].first, + delete_bitmaps_vers[i].second); + } + } + + for (int tablet_id = 800011; tablet_id <= 800015; ++tablet_id) { + // no rowsets are compacted + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + std::vector> rowset_vers {{2, 2}, {3, 3}, {4, 4}, {5, 5}, + {6, 6}, {7, 7}, {8, 8}, {9, 9}}; + std::vector> delete_bitmaps_vers { + {2, 9}, {3, 9}, {4, 9}, {5, 9}, {6, 9}, {7, 9}, {8, 9}, {9, 9}}; + std::vector segments_overlap {true, true, true, true, true, true, true, true}; + for (size_t i {0}; i < 8; i++) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + rowset_vers[i].first, rowset_vers[i].second, + rowset_id, segments_overlap[i], 1); + create_delete_bitmaps(txn.get(), tablet_id, rowset_id, delete_bitmaps_vers[i].first, + delete_bitmaps_vers[i].second); + } + } + + for (int tablet_id = 800016; tablet_id <= 800020; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + std::vector> rowset_vers { + {2, 5}, {6, 6}, {7, 7}, {8, 8}, {9, 9}}; + std::vector> delete_bitmaps_vers { + {5, 9}, {6, 9}, {7, 9}, {8, 9}, {9, 9}}; + std::vector segments_overlap {false, true, true, true, true}; + for (size_t i {0}; i < 5; i++) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + rowset_vers[i].first, rowset_vers[i].second, + rowset_id, segments_overlap[i], 1); + create_delete_bitmaps(txn.get(), tablet_id, rowset_id, delete_bitmaps_vers[i].first, + delete_bitmaps_vers[i].second); + } + } + + // also create some rowsets without delete bitmaps in non merge-on-write tablet + for (int tablet_id = 700001; tablet_id <= 700010; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, false)); + int64_t rowset_start_id = 500; + for (int ver = 2; ver < 10; ++ver) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + ver, ver, rowset_id, false, 1); + } + } + + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + ASSERT_EQ(checker.do_delete_bitmap_storage_optimize_check(), 0); +} + +TEST(CheckerTest, delete_bitmap_storage_optimize_check_abnormal) { + config::delete_bitmap_storage_optimize_check_version_gap = 0; + // abnormal case, some rowsets' delete bitmaps are not deleted as expected + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("1"); + + InstanceChecker checker(txn_kv, instance_id); + ASSERT_EQ(checker.init(instance), 0); + auto accessor = checker.accessor_map_.begin()->second; + + // tablet_id -> [rowset_id] + std::map> expected_abnormal_rowsets {}; + std::map> real_abnormal_rowsets {}; + auto sp = SyncPoint::get_instance(); + std::unique_ptr> defer( + (int*)0x01, [](int*) { SyncPoint::get_instance()->clear_all_call_backs(); }); + sp->set_call_back("InstanceChecker::check_delete_bitmap_storage_optimize.get_abnormal_rowset", + [&real_abnormal_rowsets](auto&& args) { + int64_t tablet_id = *try_any_cast(args[0]); + std::string rowset_id = *try_any_cast(args[1]); + real_abnormal_rowsets[tablet_id].insert(rowset_id); + }); + sp->enable_processing(); + + std::unique_ptr txn; + ASSERT_EQ(TxnErrorCode::TXN_OK, txn_kv->create_txn(&txn)); + + constexpr int table_id = 10000, index_id = 10001, partition_id = 10002; + + int64_t rowset_start_id = 700; + for (int tablet_id = 900001; tablet_id <= 900005; ++tablet_id) { + ASSERT_EQ(0, + create_tablet(txn_kv.get(), table_id, index_id, partition_id, tablet_id, true)); + std::vector> rowset_vers {{2, 2}, {3, 3}, {4, 4}, {5, 5}, + {6, 7}, {8, 8}, {9, 9}}; + std::vector> delete_bitmaps_vers { + {2, 9}, {7, 9}, {4, 9}, {7, 9}, {7, 9}, {8, 9}, {9, 9}}; + std::vector segments_overlap {true, true, true, true, false, true, true}; + for (size_t i {0}; i < 7; i++) { + std::string rowset_id = std::to_string(rowset_start_id++); + create_committed_rowset_with_rowset_id(txn_kv.get(), accessor.get(), "1", tablet_id, + rowset_vers[i].first, rowset_vers[i].second, + rowset_id, segments_overlap[i], 1); + create_delete_bitmaps(txn.get(), tablet_id, rowset_id, delete_bitmaps_vers[i].first, + delete_bitmaps_vers[i].second); + if (delete_bitmaps_vers[i].first < 7) { + expected_abnormal_rowsets[tablet_id].insert(rowset_id); + } + } + } + + ASSERT_EQ(TxnErrorCode::TXN_OK, txn->commit()); + + ASSERT_EQ(checker.do_delete_bitmap_storage_optimize_check(), 1); + ASSERT_EQ(expected_abnormal_rowsets, real_abnormal_rowsets); +} + TEST(RecyclerTest, delete_rowset_data) { auto txn_kv = std::make_shared(); ASSERT_EQ(txn_kv->init(), 0); @@ -2681,4 +3077,109 @@ TEST(RecyclerTest, delete_rowset_data) { } } +TEST(RecyclerTest, delete_rowset_data_without_inverted_index_storage_format) { + auto txn_kv = std::make_shared(); + ASSERT_EQ(txn_kv->init(), 0); + + InstanceInfoPB instance; + instance.set_instance_id(instance_id); + auto obj_info = instance.add_obj_info(); + obj_info->set_id("recycle_tmp_rowsets"); + obj_info->set_ak(config::test_s3_ak); + obj_info->set_sk(config::test_s3_sk); + obj_info->set_endpoint(config::test_s3_endpoint); + obj_info->set_region(config::test_s3_region); + obj_info->set_bucket(config::test_s3_bucket); + obj_info->set_prefix("recycle_tmp_rowsets"); + + std::vector schemas; + for (int i = 0; i < 5; ++i) { + auto& schema = schemas.emplace_back(); + schema.set_schema_version(i); + //schema.set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V1); + for (int j = 0; j < i; ++j) { + auto index = schema.add_index(); + index->set_index_id(j); + index->set_index_type(IndexType::INVERTED); + } + } + + { + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + int64_t txn_id_base = 114115; + int64_t tablet_id_base = 10015; + int64_t index_id_base = 1000; + // Delete each rowset directly using one RowsetPB + for (int i = 0; i < 100; ++i) { + int64_t txn_id = txn_id_base + i; + for (int j = 0; j < 20; ++j) { + auto rowset = create_rowset("recycle_tmp_rowsets", tablet_id_base + j, + index_id_base + j % 4, 5, schemas[i % 5], txn_id); + create_tmp_rowset(txn_kv.get(), accessor.get(), rowset, i & 1); + ASSERT_EQ(0, recycler.delete_rowset_data(rowset)); + } + } + + std::unique_ptr list_iter; + ASSERT_EQ(0, accessor->list_all(&list_iter)); + ASSERT_FALSE(list_iter->has_next()); + } + { + InstanceInfoPB tmp_instance; + std::string resource_id = "recycle_tmp_rowsets"; + tmp_instance.set_instance_id(instance_id); + auto tmp_obj_info = tmp_instance.add_obj_info(); + tmp_obj_info->set_id(resource_id); + tmp_obj_info->set_ak(config::test_s3_ak); + tmp_obj_info->set_sk(config::test_s3_sk); + tmp_obj_info->set_endpoint(config::test_s3_endpoint); + tmp_obj_info->set_region(config::test_s3_region); + tmp_obj_info->set_bucket(config::test_s3_bucket); + tmp_obj_info->set_prefix(resource_id); + + InstanceRecycler recycler(txn_kv, tmp_instance, thread_group, + std::make_shared(txn_kv)); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + // Delete multiple rowset files using one series of RowsetPB + constexpr int index_id = 10001, tablet_id = 10002; + std::vector rowset_pbs; + for (int i = 0; i < 10; ++i) { + auto rowset = create_rowset(resource_id, tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset( + txn_kv.get(), accessor.get(), rowset, + static_cast(i % (RecycleRowsetPB::Type_MAX + 1)), true); + + rowset_pbs.emplace_back(std::move(rowset)); + } + ASSERT_EQ(0, recycler.delete_rowset_data(rowset_pbs)); + std::unique_ptr list_iter; + ASSERT_EQ(0, accessor->list_all(&list_iter)); + ASSERT_FALSE(list_iter->has_next()); + } + { + InstanceRecycler recycler(txn_kv, instance, thread_group, + std::make_shared(txn_kv)); + ASSERT_EQ(recycler.init(), 0); + auto accessor = recycler.accessor_map_.begin()->second; + // Delete multiple rowset files using one series of RowsetPB + constexpr int index_id = 20001, tablet_id = 20002; + // Delete each rowset file directly using it's id to construct one path + for (int i = 0; i < 1000; ++i) { + auto rowset = + create_rowset("recycle_tmp_rowsets", tablet_id, index_id, 5, schemas[i % 5]); + create_recycle_rowset(txn_kv.get(), accessor.get(), rowset, RecycleRowsetPB::COMPACT, + true); + ASSERT_EQ(0, recycler.delete_rowset_data(rowset.resource_id(), rowset.tablet_id(), + rowset.rowset_id_v2())); + } + std::unique_ptr list_iter; + ASSERT_EQ(0, accessor->list_all(&list_iter)); + ASSERT_FALSE(list_iter->has_next()); + } +} + } // namespace doris::cloud diff --git a/cloud/test/s3_accessor_test.cpp b/cloud/test/s3_accessor_test.cpp index 0dd51b749d86e2..c19f5f6a1dfdfb 100644 --- a/cloud/test/s3_accessor_test.cpp +++ b/cloud/test/s3_accessor_test.cpp @@ -17,8 +17,10 @@ #include "recycler/s3_accessor.h" +#include #include #include +#include #include #include @@ -320,4 +322,70 @@ TEST(S3AccessorTest, gcs) { test_s3_accessor(*accessor); } +TEST(S3AccessorTest, path_style_test) { + ObjectStoreInfoPB obj_info; + obj_info.set_prefix("doris-debug-instance-prefix"); + obj_info.set_provider(ObjectStoreInfoPB_Provider_S3); + obj_info.set_ak("dummy_ak"); + obj_info.set_sk("dummy_sk"); + obj_info.set_endpoint("dummy-bucket"); + obj_info.set_region("cn-north-1"); + obj_info.set_bucket("dummy-bucket"); + config::max_s3_client_retry = 0; + + auto* sp = SyncPoint::get_instance(); + sp->enable_processing(); + std::vector guards; + + std::string base_domain = "s3.cn-north-1.amazonaws.com.cn"; + std::string domain_ip = "54.222.51.71"; // first ip of base_domain + // to test custom_domain, add ${domain_ip} ${custom_domain} to /etc/hosts + // otherwise the related cases will fail + std::string custom_domain = "gavin.s3.aws.com"; + // clang-format off + // http code 403 means there is nothing wrong the given domain in objinfo + // domain, use_path_style, http_code + std::vector> inputs { + {base_domain , false , 403}, // works + {base_domain , true , 403}, // works + {"http://" + base_domain , false , 403}, // works + {"http://" + base_domain , true , 403}, // works + {"https://" + base_domain , false , 403}, // works + {"https://" + base_domain , true , 403}, // works + {"http://" + domain_ip , false , 301}, // works, ip with virtual addressing + {"http://" + domain_ip , true , 301}, // works, ip with path style + {custom_domain , false , -1} , // custom_domain could not resolve with virtual addressing + {custom_domain , true , 403}, // custom_domain working with path style + {"http://" + custom_domain , false , -1} , // custom_domain could not resolve with virtual addressing + {"https://" + custom_domain, true , -1}, // certificate issue, custom_domain does not attached with any certs + // {"https://54.222.51.71" , false , -1} , // certificate issue + // {"https://54.222.51.71" , true , -1} , // certificate issue + }; + + int case_idx = 0; + sp->set_call_back("S3ObjClient::delete_object", + [&case_idx, &inputs](auto&& args) { + auto* res = try_any_cast(args[0]); + EXPECT_EQ(std::get<2>(inputs[case_idx]), static_cast(res->GetError().GetResponseCode())) << "<<<<<<<<<<<<<<<<<<<<< " << case_idx; + case_idx++; + }, + &guards.emplace_back()); + // clang-format on + + for (auto& i : inputs) { + obj_info.set_endpoint(std::get<0>(i)); + obj_info.set_use_path_style(std::get<1>(i)); + auto s3_conf = S3Conf::from_obj_store_info(obj_info); + EXPECT_EQ(s3_conf->use_virtual_addressing, !obj_info.use_path_style()) << case_idx; + std::shared_ptr accessor; + int ret = S3Accessor::create(*s3_conf, &accessor); + EXPECT_EQ(ret, 0) << "<<<<<<<<<<<<<<<<<<<<< " << case_idx; + ret = accessor->init(); + EXPECT_EQ(ret, 0) << "<<<<<<<<<<<<<<<<<<<<< " << case_idx; + // this function call will trigger syncpoint callback to increment case_idx + accessor->delete_file("abc"); // try to delete a nonexisted file, ignore the result + // EXPECT_EQ(ret, exp) << "<<<<<<<<<<<<<<<<<<<<< " << case_idx << " domain " << std::get<0>(i); + } +} + } // namespace doris::cloud diff --git a/cloud/test/txn_lazy_commit_test.cpp b/cloud/test/txn_lazy_commit_test.cpp index 9a7679f3dd9e23..0f284508a3f34e 100644 --- a/cloud/test/txn_lazy_commit_test.cpp +++ b/cloud/test/txn_lazy_commit_test.cpp @@ -25,7 +25,9 @@ #include #include +#include #include +#include #include #include #include @@ -1812,4 +1814,90 @@ TEST(TxnLazyCommitTest, ConcurrentCommitTxnEventuallyCase4Test) { ASSERT_TRUE(abort_timeout_txn_hit); ASSERT_EQ(txn_id, txn_info_pb.txn_id()); } -} // namespace doris::cloud \ No newline at end of file + +TEST(TxnLazyCommitTest, RowsetMetaSizeExceedTest) { + auto txn_kv = get_mem_txn_kv(); + + int64_t db_id = 5252025; + int64_t table_id = 35201043384; + int64_t index_id = 256439; + int64_t partition_id = 732536259; + + auto meta_service = get_meta_service(txn_kv, true); + int64_t tablet_id = 25910248; + + { + create_tablet_with_db_id(meta_service.get(), db_id, table_id, index_id, partition_id, + tablet_id); + } + { + int tmp_txn_id = 0; + { + brpc::Controller cntl; + BeginTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + TxnInfoPB txn_info_pb; + txn_info_pb.set_db_id(db_id); + txn_info_pb.set_label("test_label_32ae213dasg3"); + txn_info_pb.add_table_ids(table_id); + txn_info_pb.set_timeout_ms(36000); + req.mutable_txn_info()->CopyFrom(txn_info_pb); + BeginTxnResponse res; + meta_service->begin_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + tmp_txn_id = res.txn_id(); + ASSERT_GT(res.txn_id(), 0); + } + { + auto tmp_rowset = create_rowset(tmp_txn_id, tablet_id, partition_id); + CreateRowsetResponse res; + commit_rowset(meta_service.get(), tmp_rowset, res); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + { + brpc::Controller cntl; + CommitTxnRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + req.set_db_id(db_id); + req.set_txn_id(tmp_txn_id); + req.set_is_2pc(false); + req.set_enable_txn_lazy_commit(true); + CommitTxnResponse res; + meta_service->commit_txn(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), + &req, &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::OK); + } + } + + auto* sp = SyncPoint::get_instance(); + sp->set_call_back("get_rowset:meta_exceed_limit", [](auto&& args) { + auto* byte_size = try_any_cast(args[0]); + *byte_size = std::numeric_limits::max(); + ++(*byte_size); + }); + + sp->enable_processing(); + { + brpc::Controller cntl; + GetRowsetRequest req; + req.set_cloud_unique_id("test_cloud_unique_id"); + auto* tablet_idx = req.mutable_idx(); + tablet_idx->set_table_id(table_id); + tablet_idx->set_index_id(index_id); + tablet_idx->set_partition_id(partition_id); + tablet_idx->set_tablet_id(tablet_id); + req.set_start_version(0); + req.set_end_version(-1); + req.set_cumulative_compaction_cnt(0); + req.set_base_compaction_cnt(0); + req.set_cumulative_point(2); + + GetRowsetResponse res; + meta_service->get_rowset(reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, + &res, nullptr); + ASSERT_EQ(res.status().code(), MetaServiceCode::PROTOBUF_PARSE_ERR); + } +} + +} // namespace doris::cloud diff --git a/conf/be.conf b/conf/be.conf index fc89b5985b2454..6480402360eb6e 100644 --- a/conf/be.conf +++ b/conf/be.conf @@ -24,7 +24,7 @@ LOG_DIR="${DORIS_HOME}/log/" JAVA_OPTS="-Dfile.encoding=UTF-8 -Xmx2048m -DlogPath=$LOG_DIR/jni.log -Xloggc:$LOG_DIR/be.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives" # For jdk 17, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Dfile.encoding=UTF-8 -Xmx2048m -DlogPath=$LOG_DIR/jni.log -Xlog:gc*:$LOG_DIR/be.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED" +JAVA_OPTS_FOR_JDK_17="-Dfile.encoding=UTF-8 -Djol.skipHotspotSAAttach=true -Xmx2048m -DlogPath=$LOG_DIR/jni.log -Xlog:gc*:$LOG_DIR/be.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED -Darrow.enable_null_check_for_get=false" # Set your own JAVA_HOME # JAVA_HOME=/path/to/jdk/ diff --git a/conf/fe.conf b/conf/fe.conf index bbfad0c15a82d0..72734a86733a31 100644 --- a/conf/fe.conf +++ b/conf/fe.conf @@ -27,10 +27,10 @@ CUR_DATE=`date +%Y%m%d-%H%M%S` LOG_DIR = ${DORIS_HOME}/log # For jdk 8 -JAVA_OPTS="-Dfile.encoding=UTF-8 -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$LOG_DIR/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true" +JAVA_OPTS="-Dfile.encoding=UTF-8 -Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx8192m -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+PrintClassHistogramAfterFullGC -Xloggc:$LOG_DIR/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true" # For jdk 17, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Dfile.encoding=UTF-8 -Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$LOG_DIR -Xlog:gc*:$LOG_DIR/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED" +JAVA_OPTS_FOR_JDK_17="-Dfile.encoding=UTF-8 -Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$LOG_DIR -Xlog:gc*,classhisto*=trace:$LOG_DIR/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED" # Set your own JAVA_HOME # JAVA_HOME=/path/to/jdk/ diff --git a/docker/runtime/doris-compose/Readme.md b/docker/runtime/doris-compose/Readme.md index c4c4dc0990f0fc..fea13ab1821882 100644 --- a/docker/runtime/doris-compose/Readme.md +++ b/docker/runtime/doris-compose/Readme.md @@ -32,6 +32,8 @@ docker run hello-world if have problem with permission denied, then [add-docker-permission](https://docs.docker.com/engine/install/linux-postinstall/). +Make sure BuildKit configured in the machine. if not follow [docker-with-BuildKit](https://docs.docker.com/build/buildkit/). + ##### 2. The doris image should contains ``` diff --git a/docker/runtime/doris-compose/cluster.py b/docker/runtime/doris-compose/cluster.py index ba834167bd1c63..f4522181d4b18e 100644 --- a/docker/runtime/doris-compose/cluster.py +++ b/docker/runtime/doris-compose/cluster.py @@ -23,6 +23,7 @@ import os import os.path import utils +import time DOCKER_DORIS_PATH = "/opt/apache-doris" LOCAL_DORIS_PATH = os.getenv("LOCAL_DORIS_PATH", "/tmp/doris") @@ -139,11 +140,15 @@ def gen_subnet_prefix16(): def get_master_fe_endpoint(cluster_name): master_fe_ip_file = get_cluster_path(cluster_name) + "/status/master_fe_ip" - if os.path.exists(master_fe_ip_file): - with open(master_fe_ip_file, "r") as f: - return "{}:{}".format(f.read().strip(), FE_QUERY_PORT) + max_retries = 10 + for attempt in range(max_retries): + if os.path.exists(master_fe_ip_file): + with open(master_fe_ip_file, "r") as f: + return "{}:{}".format(f.read().strip(), FE_QUERY_PORT) + time.sleep(1) try: cluster = Cluster.load(cluster_name) + LOG.info("master file not exist, master ip get from node 1") return "{}:{}".format( cluster.get_node(Node.TYPE_FE, 1).get_ip(), FE_QUERY_PORT) except: @@ -468,6 +473,7 @@ def get_add_init_config(self): for key in ("JAVA_OPTS", "JAVA_OPTS_FOR_JDK_17"): value = parser["dummy_section"].get(key) if value: + value = value.strip().strip('"') cfg.append( f"{key} = \"{value} -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:{FE_JAVA_DBG_PORT}\"" ) diff --git a/docker/runtime/doris-compose/command.py b/docker/runtime/doris-compose/command.py index 7a2f3f3c195f18..638c1c465d75b1 100644 --- a/docker/runtime/doris-compose/command.py +++ b/docker/runtime/doris-compose/command.py @@ -183,7 +183,7 @@ def _support_boolean_action(self): return sys.version_info.major == 3 and sys.version_info.minor >= 9 def _print_table(self, header, datas): - if utils.is_enable_log(): + if utils.is_log_stdout(): table = prettytable.PrettyTable( [utils.render_green(field) for field in header]) for row in datas: @@ -598,13 +598,6 @@ def do_add_node(node_type, add_num, add_ids): related_nodes, output_real_time=output_real_time) - ls_cmd = "python docker/runtime/doris-compose/doris-compose.py ls " + cluster.name - LOG.info("Inspect command: " + utils.render_green(ls_cmd) + "\n") - LOG.info( - "Master fe query address: " + - utils.render_green(CLUSTER.get_master_fe_endpoint(cluster.name)) + - "\n") - if not args.start: LOG.info( utils.render_green( @@ -618,14 +611,18 @@ def do_add_node(node_type, add_num, add_ids): LOG.info("Waiting for FE master to be elected...") expire_ts = time.time() + 30 while expire_ts > time.time(): + ready = False db_mgr = database.get_db_mgr(args.NAME, False) for id in add_fe_ids: fe_state = db_mgr.get_fe(id) if fe_state is not None and fe_state.alive: + ready = True break - LOG.info("there is no fe ready") - time.sleep(5) - + if ready: + break + LOG.info("there is no fe ready") + time.sleep(1) + LOG.info("after Waiting for FE master to be elected...") if cluster.is_cloud and args.sql_mode_node_mgr: db_mgr = database.get_db_mgr(args.NAME, False) master_fe_endpoint = CLUSTER.get_master_fe_endpoint( @@ -635,7 +632,9 @@ def do_add_node(node_type, add_num, add_ids): fe_endpoint = f"{fe.get_ip()}:{CLUSTER.FE_EDITLOG_PORT}" if fe_endpoint != master_fe_endpoint: try: - db_mgr.add_fe(fe_endpoint) + db_mgr.add_fe( + fe_endpoint, "FOLLOWER" + if cluster.fe_follower else "OBSERVER") LOG.info(f"Added FE {fe_endpoint} successfully.") except Exception as e: LOG.error( @@ -661,6 +660,12 @@ def do_add_node(node_type, add_num, add_ids): "Up cluster {} succ, related node num {}".format( args.NAME, related_node_num))) + ls_cmd = "python docker/runtime/doris-compose/doris-compose.py ls " + cluster.name + LOG.info("Inspect command: " + utils.render_green(ls_cmd) + "\n") + LOG.info( + "Master fe query address: " + + utils.render_green(CLUSTER.get_master_fe_endpoint(cluster.name)) + + "\n") return { "fe": { "add_list": add_fe_ids, @@ -1066,8 +1071,7 @@ def parse_cluster_compose_file(cluster_name): if services is None: return COMPOSE_BAD, {} return COMPOSE_GOOD, { - service: - ComposeService( + service: ComposeService( service, list(service_conf["networks"].values())[0] ["ipv4_address"], service_conf["image"]) diff --git a/docker/runtime/doris-compose/database.py b/docker/runtime/doris-compose/database.py index 46cdd961c9f888..370f1d5ee2afa7 100644 --- a/docker/runtime/doris-compose/database.py +++ b/docker/runtime/doris-compose/database.py @@ -27,12 +27,13 @@ class FEState(object): - def __init__(self, id, is_master, alive, last_heartbeat, err_msg): + def __init__(self, id, is_master, alive, last_heartbeat, err_msg, role): self.id = id self.is_master = is_master self.alive = alive self.last_heartbeat = last_heartbeat self.err_msg = err_msg + self.role = role class BEState(object): @@ -66,11 +67,11 @@ def load_states(self): self._load_fe_states() self._load_be_states() - def add_fe(self, fe_endpoint): + def add_fe(self, fe_endpoint, role): try: - sql = f"ALTER SYSTEM ADD FOLLOWER '{fe_endpoint}'" + sql = f"ALTER SYSTEM ADD {role} '{fe_endpoint}'" self._exec_query(sql) - LOG.info(f"Added FE {fe_endpoint} via SQL successfully.") + LOG.info(f"Added {role} FE {fe_endpoint} via SQL successfully.") except Exception as e: LOG.error(f"Failed to add FE {fe_endpoint} via SQL: {str(e)}") raise @@ -78,8 +79,9 @@ def add_fe(self, fe_endpoint): def drop_fe(self, fe_endpoint): id = CLUSTER.Node.get_id_from_ip(fe_endpoint[:fe_endpoint.find(":")]) try: - self._exec_query( - "ALTER SYSTEM DROP FOLLOWER '{}'".format(fe_endpoint)) + role = self.get_fe(id).role if self.get_fe(id) else "FOLLOWER" + self._exec_query("ALTER SYSTEM DROP {} '{}'".format( + role, fe_endpoint)) LOG.info("Drop fe {} with id {} from db succ.".format( fe_endpoint, id)) except Exception as e: @@ -152,7 +154,7 @@ def decommission_be(self, be_endpoint): .format(be_endpoint, be.alive, be.decommissioned, be.tablet_num, old_tablet_num, int(time.time() - start_ts))) - time.sleep(5) + time.sleep(1) def create_default_storage_vault(self, cloud_store_config): try: @@ -194,7 +196,7 @@ def _load_fe_states(self): id = CLUSTER.Node.get_id_from_ip(ip) last_heartbeat = utils.escape_null(record["LastHeartbeat"]) err_msg = record["ErrMsg"] - fe = FEState(id, is_master, alive, last_heartbeat, err_msg) + fe = FEState(id, is_master, alive, last_heartbeat, err_msg, role) fe_states[id] = fe if is_master and alive: alive_master_fe_ip = ip @@ -223,13 +225,23 @@ def _load_be_states(self): self.be_states = be_states # return rows, and each row is a record map - def _exec_query(self, sql): + def _exec_query(self, sql, retries=3): self._prepare_conn() - with self.conn.cursor() as cursor: - cursor.execute(sql) - fields = [field_md[0] for field_md in cursor.description - ] if cursor.description else [] - return [dict(zip(fields, row)) for row in cursor.fetchall()] + for attempt in range(retries): + try: + with self.conn.cursor() as cursor: + cursor.execute(sql) + fields = [field_md[0] for field_md in cursor.description + ] if cursor.description else [] + return [dict(zip(fields, row)) for row in cursor.fetchall()] + except Exception as e: + LOG.warn(f"Error occurred: {e}") + if "timed out" in str(e).lower() and attempt < retries - 1: + LOG.warn(f"Query timed out. Retrying {attempt + 1}/{retries}...") + self._reset_conn() + else: + raise e + raise Exception("Max retries exceeded") def _prepare_conn(self): if self.conn: @@ -257,19 +269,23 @@ def get_db_mgr(cluster_name, required_load_succ=True): if not master_fe_ip: return db_mgr - has_alive_fe = False + alive_fe = None + cluster = CLUSTER.Cluster.load(cluster_name) containers = utils.get_doris_containers(cluster_name).get(cluster_name, []) for container in containers: if utils.is_container_running(container): - _, node_type, _ = utils.parse_service_name(container.name) + _, node_type, id = utils.parse_service_name(container.name) if node_type == CLUSTER.Node.TYPE_FE: - has_alive_fe = True - break - - if not has_alive_fe: + node = cluster.get_node(node_type, id) + if not alive_fe: + alive_fe = node + if node.get_ip() == master_fe_ip: + alive_fe = node + break + if not alive_fe: return db_mgr - db_mgr.master_fe_ip = master_fe_ip + db_mgr.master_fe_ip = alive_fe.get_ip() try: db_mgr.load_states() except Exception as e: diff --git a/docker/runtime/doris-compose/doris-compose.py b/docker/runtime/doris-compose/doris-compose.py index a2d3a517553f71..cf3692d53215e0 100644 --- a/docker/runtime/doris-compose/doris-compose.py +++ b/docker/runtime/doris-compose/doris-compose.py @@ -16,7 +16,9 @@ # under the License. import argparse +import cluster as CLUSTER import command +import os.path import sys import traceback import utils @@ -31,12 +33,12 @@ def parse_args(): return ap.parse_args(), ap.format_help() -def run(args, disable_log, help): +def run(args, disable_log_stdout, help): for cmd in command.ALL_COMMANDS: if args.command == cmd.name: timer = utils.Timer() result = cmd.run(args) - if cmd.print_use_time() and not disable_log: + if cmd.print_use_time() and not disable_log_stdout: timer.show() return result print(help) @@ -48,19 +50,26 @@ def run(args, disable_log, help): verbose = getattr(args, "verbose", False) if verbose: utils.set_log_verbose() - disable_log = getattr(args, "output_json", False) - if disable_log: - utils.set_enable_log(False) + disable_log_stdout = getattr(args, "output_json", False) + if disable_log_stdout: + log_file_name = "" + cluster_name = getattr(args, "NAME", "") + if cluster_name: + if type(cluster_name) == type([]): + cluster_name = cluster_name[0] + log_file_name = os.path.join( + CLUSTER.get_cluster_path(cluster_name), "doris-compose.log") + utils.set_log_to(log_file_name, False) code = None try: - data = run(args, disable_log, help) - if disable_log: + data = run(args, disable_log_stdout, help) + if disable_log_stdout: print(utils.pretty_json({"code": 0, "data": data}), flush=True) code = 0 except: err = traceback.format_exc() - if disable_log: + if disable_log_stdout: print(utils.pretty_json({"code": 1, "err": err}), flush=True) else: print(err, flush=True) diff --git a/docker/runtime/doris-compose/requirements.txt b/docker/runtime/doris-compose/requirements.txt index 2f962ed68d8bf8..1f32223a02e1f4 100644 --- a/docker/runtime/doris-compose/requirements.txt +++ b/docker/runtime/doris-compose/requirements.txt @@ -22,5 +22,6 @@ jsonpickle prettytable pymysql python-dateutil -#pyyaml==5.4.1 +# if mac install pyyaml failed, change pyyaml version +#pyyaml==5.3.1 requests<=2.31.0 diff --git a/docker/runtime/doris-compose/resource/init_fe.sh b/docker/runtime/doris-compose/resource/init_fe.sh index b69ac3a209e409..a58723db1d7b2a 100755 --- a/docker/runtime/doris-compose/resource/init_fe.sh +++ b/docker/runtime/doris-compose/resource/init_fe.sh @@ -102,9 +102,6 @@ start_cloud_fe() { fe_daemon & run_fe - if [ "$MY_ID" == "1" ]; then - echo $MY_IP >$MASTER_FE_IP_FILE - fi return fi @@ -168,10 +165,6 @@ start_cloud_fe() { fe_daemon & run_fe - - if [ "$MY_ID" == "1" ]; then - echo $MY_IP >$MASTER_FE_IP_FILE - fi } stop_frontend() { diff --git a/docker/runtime/doris-compose/utils.py b/docker/runtime/doris-compose/utils.py index 4332ae6cf48d03..dcb821ddffdd00 100644 --- a/docker/runtime/doris-compose/utils.py +++ b/docker/runtime/doris-compose/utils.py @@ -23,6 +23,7 @@ import pwd import socket import subprocess +import sys import time import yaml @@ -30,7 +31,7 @@ LOG = None -ENABLE_LOG = True +ENALBE_LOG_STDOUT = True class Timer(object): @@ -48,39 +49,41 @@ def cancel(self): self.canceled = True -def set_enable_log(enabled): - global ENABLE_LOG - ENABLE_LOG = enabled - get_logger().disabled = not enabled - - -def is_enable_log(): - return ENABLE_LOG +def is_log_stdout(): + return ENALBE_LOG_STDOUT def set_log_verbose(): get_logger().setLevel(logging.DEBUG) -def get_logger(name=None): - global LOG - if LOG != None: - return LOG - - logger = logging.getLogger(name) - if not logger.hasHandlers(): +def set_log_to(log_file_name, is_to_stdout): + logger = get_logger() + for ch in logger.handlers: + logger.removeHandler(ch) + if log_file_name: + os.makedirs(os.path.dirname(log_file_name), exist_ok=True) + logger.addHandler(logging.FileHandler(log_file_name)) + global ENALBE_LOG_STDOUT + ENALBE_LOG_STDOUT = is_to_stdout + if is_to_stdout: + logger.addHandler(logging.StreamHandler(sys.stdout)) + for ch in logger.handlers: formatter = logging.Formatter( '%(asctime)s - %(filename)s - %(lineno)dL - %(levelname)s - %(message)s' ) - ch = logging.StreamHandler() ch.setLevel(logging.DEBUG) ch.setFormatter(formatter) - logger.addHandler(ch) - logger.setLevel(logging.INFO) - LOG = logger - return logger +def get_logger(name="doris-compose"): + global LOG + if LOG is None: + LOG = logging.getLogger(name) + LOG.setLevel(logging.INFO) + set_log_to(None, True) + + return LOG get_logger() @@ -196,15 +199,17 @@ def exec_shell_command(command, ignore_errors=False, output_real_time=False): if output_real_time: while p.poll() is None: s = p.stdout.readline().decode('utf-8') - if ENABLE_LOG and s.rstrip(): - print(s.rstrip()) + if s.rstrip(): + for line in s.strip().splitlines(): + LOG.info("(docker) " + line) out += s exitcode = p.wait() else: out = p.communicate()[0].decode('utf-8') exitcode = p.returncode - if ENABLE_LOG and out: - print(out) + if out: + for line in out.splitlines(): + LOG.info("(docker) " + line) if not ignore_errors: assert exitcode == 0, out return exitcode, out diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql new file mode 100644 index 00000000000000..adf0f7d56b27d9 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run69.hql @@ -0,0 +1,35 @@ +use `default`; + + +CREATE TABLE json_nested_complex_table ( + user_ID STRING, + user_PROFILE STRUCT< + name: STRING, + AGE: INT, + preferences: MAP< + STRING, + STRUCT< + preference_ID: INT, + preference_VALUES: ARRAY + > + > + >, + activity_LOG ARRAY< + STRUCT< + activity_DATE: STRING, + activities: MAP< + STRING, + STRUCT< + `DETAILS`: STRING, + metrics: MAP + > + > + > + > +) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' + +LOCATION + '/user/doris/preinstalled_data/json/json_nested_complex_table'; + + +msck repair table json_nested_complex_table; diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql new file mode 100644 index 00000000000000..73df8cba557bcb --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run70.hql @@ -0,0 +1,73 @@ +use `default`; + + +CREATE TABLE json_all_complex_types ( + `id` int, + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `binary_col` binary, + `date_col` date, + `timestamp_col1` timestamp, + `timestamp_col2` timestamp, + `timestamp_col3` timestamp, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array + ) PARTITIONED BY (`dt` string) +ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' +LOCATION + '/user/doris/preinstalled_data/json/json_all_complex_types'; + +msck repair table json_all_complex_types; diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql new file mode 100644 index 00000000000000..ec99e72d2f5780 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run71.hql @@ -0,0 +1,13 @@ +use `default`; + + +CREATE TABLE json_load_data_table ( + `id` int, + `col1` int, + `col2` struct< col2a:int, col2b:string>, + `col3` map +) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' +LOCATION + '/user/doris/preinstalled_data/json/json_load_data_table'; + +msck repair table json_load_data_table; diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 new file mode 100644 index 00000000000000..5fe37cbc6f098e --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt1/000000_0 @@ -0,0 +1,3 @@ +{"id":1,"boolean_col":true,"tinyint_col":127,"smallint_col":32767,"int_col":2147483647,"bigint_col":9223372036854775807,"float_col":123.45,"double_col":123456.789,"decimal_col1":123456789,"decimal_col2":1234.5678,"decimal_col3":123456.789012,"decimal_col4":123456789.012345678901,"string_col":"string_value","binary_col":"binary_value","date_col":"2024-03-20","timestamp_col1":"2024-03-20 12:00:00","timestamp_col2":"2024-03-20 12:00:00.123456789","timestamp_col3":"2024-03-20 12:00:00.123456789","char_col1":"char_value1 ","char_col2":"char_value2 ","char_col3":"char_value3 ","varchar_col1":"varchar_value1","varchar_col2":"varchar_value2","varchar_col3":"varchar_value3","t_map_string":{"key1":"value1"},"t_map_varchar":{"key1":"value1"},"t_map_char":{"a ":"b "},"t_map_int":{"1":10},"t_map_bigint":{"1":100000000000},"t_map_float":{"1.1":10.1},"t_map_double":{"1.1":10.1},"t_map_boolean":{"true":false},"t_map_decimal_precision_2":{"1.1":1.1},"t_map_decimal_precision_4":{"1.23":1.23},"t_map_decimal_precision_8":{"1.2345":1.2345},"t_map_decimal_precision_17":{"1.23456789":1.23456789},"t_map_decimal_precision_18":{"1.23456789":1.23456789},"t_map_decimal_precision_38":{"1.2345678901234568":1.2345678901234568},"t_array_string":["string1","string2"],"t_array_int":[1,2,3],"t_array_bigint":[100000000000,200000000000],"t_array_float":[1.1,2.2],"t_array_double":[1.123456789,2.123456789],"t_array_boolean":[true,false],"t_array_varchar":["varchar1","varchar2"],"t_array_char":["char1 ","char2 "],"t_array_decimal_precision_2":[1.1,2.2],"t_array_decimal_precision_4":[1.23,2.34],"t_array_decimal_precision_8":[1.2345,2.3456],"t_array_decimal_precision_17":[1.23456789,2.34567891],"t_array_decimal_precision_18":[1.23456789,2.34567891],"t_array_decimal_precision_38":[1.2345678901234568,2.3456789012345679],"t_struct_bigint":{"s_bigint":1234567890},"t_complex":{"key":[{"s_int":123}]},"t_struct_nested":{"struct_field":["value1","value2"]},"t_struct_null":{"struct_field_null":null,"struct_field_null2":null},"t_struct_non_nulls_after_nulls":{"struct_non_nulls_after_nulls1":123,"struct_non_nulls_after_nulls2":"value"},"t_nested_struct_non_nulls_after_nulls":{"struct_field1":123,"struct_field2":"value","strict_field3":{"nested_struct_field1":123,"nested_struct_field2":"nested_value"}},"t_map_null_value":{"null_key":null},"t_array_string_starting_with_nulls":[null,"value1","value2"],"t_array_string_with_nulls_in_between":["value1",null,"value2"],"t_array_string_ending_with_nulls":["value1","value2",null],"t_array_string_all_nulls":[null,null,null]} +{"id":2,"boolean_col":false,"tinyint_col":58,"smallint_col":12345,"int_col":2147483000,"bigint_col":null,"float_col":789.56,"double_col":654321.123,"decimal_col1":987654321,"decimal_col2":5678.1234,"decimal_col3":987654.321098,"decimal_col4":987654321.098765432109,"string_col":"changed_string","binary_col":"new_binary_value","date_col":"2025-05-25","timestamp_col1":"2025-05-25 15:30:00","timestamp_col2":"2025-05-25 15:30:00.654321987","timestamp_col3":"2025-05-25 15:30:00.654321987","char_col1":"char_new_value1 ","char_col2":"char_new_value2 ","char_col3":"char_new_value3 ","varchar_col1":"varchar_new_value1","varchar_col2":"varchar_new_value2","varchar_col3":"varchar_new_value3","t_map_string":{"key2":"value2"},"t_map_varchar":{"key2":"value2"},"t_map_char":{"x ":"y "},"t_map_int":{"2":20},"t_map_bigint":{"2":200000000000},"t_map_float":{"2.2":20.2},"t_map_double":{"2.2":20.2},"t_map_boolean":{"false":true},"t_map_decimal_precision_2":{"2.2":2.2},"t_map_decimal_precision_4":{"2.34":2.34},"t_map_decimal_precision_8":{"2.3456":2.3456},"t_map_decimal_precision_17":{"2.34567891":2.34567891},"t_map_decimal_precision_18":{"2.34567891":2.34567891},"t_map_decimal_precision_38":{"2.3456789012345679":2.3456789012345679},"t_array_string":["string3","string4"],"t_array_int":[4,5,6],"t_array_bigint":[300000000000,400000000000],"t_array_float":[2.2,3.3],"t_array_double":[2.123456789,3.123456789],"t_array_boolean":[false,true],"t_array_varchar":["varchar3","varchar4"],"t_array_char":["char3 ","char4 "],"t_array_decimal_precision_2":[2.2,3.3],"t_array_decimal_precision_4":[2.34,3.45],"t_array_decimal_precision_8":[2.3456,3.4567],"t_array_decimal_precision_17":[2.34567891,3.45678901],"t_array_decimal_precision_18":[2.34567891,3.45678901],"t_array_decimal_precision_38":[2.3456789012345679,3.4567890123456789],"t_struct_bigint":{"s_bigint":9876543210},"t_complex":{"key2":[{"s_int":456}]},"t_struct_nested":{"struct_field":["new_value1","new_value2"]},"t_struct_null":{"struct_field_null":null,"struct_field_null2":null},"t_struct_non_nulls_after_nulls":{"struct_non_nulls_after_nulls1":456,"struct_non_nulls_after_nulls2":"new_value"},"t_nested_struct_non_nulls_after_nulls":{"struct_field1":456,"struct_field2":"new_value","strict_field3":{"nested_struct_field1":456,"nested_struct_field2":"nested_value2"}},"t_map_null_value":{"null_key":null},"t_array_string_starting_with_nulls":[null,"new_value1","new_value2"],"t_array_string_with_nulls_in_between":["new_value1",null,"new_value2"],"t_array_string_ending_with_nulls":["new_value1","new_value2",null],"t_array_string_all_nulls":[null,null,null]} +{"id":3,"boolean_col":false,"tinyint_col":-128,"smallint_col":-32768,"int_col":-2147483648,"bigint_col":-9223372036854775808,"float_col":-3.4028235E38,"double_col":-1.7976931348623157E308,"decimal_col1":-999999999,"decimal_col2":-9999.9999,"decimal_col3":-999999999.999999,"decimal_col4":null,"string_col":"min_string_value","binary_col":"xxxx","date_col":"2001-01-01","timestamp_col1":"2001-01-01 00:00:00","timestamp_col2":"2001-01-01 00:00:00","timestamp_col3":"2001-01-01 00:00:00","char_col1":"char_min_value1 ","char_col2":"char_min_value2 ","char_col3":"char_min_value3 ","varchar_col1":"varchar_min_value1","varchar_col2":"varchar_min_value2","varchar_col3":"varchar_min_value3","t_map_string":{"min_key":"min_value"},"t_map_varchar":{"min_key":"min_value"},"t_map_char":{"a ":"z "},"t_map_int":{"-1":-100},"t_map_bigint":{"-1":-100000000000},"t_map_float":{"-1.1":-10.1},"t_map_double":{"-1.1":-10.1},"t_map_boolean":{"false":true},"t_map_decimal_precision_2":{"-1.1":-1.1},"t_map_decimal_precision_4":{"-1.23":-1.23},"t_map_decimal_precision_8":{"-1.2345":-1.2345},"t_map_decimal_precision_17":{"-1.23456789":-1.23456789},"t_map_decimal_precision_18":{"-1.23456789":-1.23456789},"t_map_decimal_precision_38":{"-1.2345678901234568":-1.2345678901234568},"t_array_string":["min_string1","min_string2"],"t_array_int":[-10,-5,-3],"t_array_bigint":[-100000000000,-200000000000],"t_array_float":[-1.1,-2.2],"t_array_double":[-1.123456789,-2.123456789],"t_array_boolean":[false,true],"t_array_varchar":["min_varchar1","min_varchar2"],"t_array_char":["min_char1 ","min_char2 "],"t_array_decimal_precision_2":[-1.1,-2.2],"t_array_decimal_precision_4":[-1.23,-2.34],"t_array_decimal_precision_8":[-1.2345,-2.3456],"t_array_decimal_precision_17":[-1.23456789,-2.34567891],"t_array_decimal_precision_18":[-1.23456789,-2.34567891],"t_array_decimal_precision_38":[-1.2345678901234568,-2.3456789012345679],"t_struct_bigint":{"s_bigint":-1234567890},"t_complex":{"min_key":[{"s_int":-123}]},"t_struct_nested":{"struct_field":["min_value1","min_value2"]},"t_struct_null":{"struct_field_null":null,"struct_field_null2":null},"t_struct_non_nulls_after_nulls":{"struct_non_nulls_after_nulls1":-123,"struct_non_nulls_after_nulls2":"min_value"},"t_nested_struct_non_nulls_after_nulls":{"struct_field1":-123,"struct_field2":"min_value","strict_field3":{"nested_struct_field1":-123,"nested_struct_field2":"nested_value"}},"t_map_null_value":{"null_key":null},"t_array_string_starting_with_nulls":[null,"min_value1","min_value2"],"t_array_string_with_nulls_in_between":["min_value1",null,"min_value2"],"t_array_string_ending_with_nulls":["min_value1","min_value2",null],"t_array_string_all_nulls":[null,null,null]} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 new file mode 100644 index 00000000000000..0a823bee693d76 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt2/000000_0 @@ -0,0 +1 @@ +{"id":4,"boolean_col":null,"tinyint_col":null,"smallint_col":null,"int_col":null,"bigint_col":null,"float_col":123.45,"double_col":null,"decimal_col1":null,"decimal_col2":null,"decimal_col3":null,"decimal_col4":null,"string_col":null,"binary_col":null,"date_col":null,"timestamp_col1":null,"timestamp_col2":null,"timestamp_col3":null,"char_col1":null,"char_col2":null,"char_col3":null,"varchar_col1":null,"varchar_col2":null,"varchar_col3":null,"t_map_string":null,"t_map_varchar":null,"t_map_char":null,"t_map_int":{"1":10},"t_map_bigint":null,"t_map_float":null,"t_map_double":null,"t_map_boolean":null,"t_map_decimal_precision_2":null,"t_map_decimal_precision_4":null,"t_map_decimal_precision_8":null,"t_map_decimal_precision_17":null,"t_map_decimal_precision_18":null,"t_map_decimal_precision_38":null,"t_array_string":null,"t_array_int":null,"t_array_bigint":null,"t_array_float":null,"t_array_double":null,"t_array_boolean":null,"t_array_varchar":null,"t_array_char":null,"t_array_decimal_precision_2":null,"t_array_decimal_precision_4":null,"t_array_decimal_precision_8":[1.2345,2.3456],"t_array_decimal_precision_17":null,"t_array_decimal_precision_18":null,"t_array_decimal_precision_38":null,"t_struct_bigint":null,"t_complex":null,"t_struct_nested":null,"t_struct_null":null,"t_struct_non_nulls_after_nulls":null,"t_nested_struct_non_nulls_after_nulls":null,"t_map_null_value":null,"t_array_string_starting_with_nulls":[null,"value1","value2"],"t_array_string_with_nulls_in_between":null,"t_array_string_ending_with_nulls":null,"t_array_string_all_nulls":null} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 new file mode 100644 index 00000000000000..a5e46399fdd553 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_all_complex_types/dt=dt3/000000_0 @@ -0,0 +1,2 @@ +{"id":5,"boolean_col":null,"tinyint_col":null,"smallint_col":null,"int_col":null,"bigint_col":null,"float_col":null,"double_col":null,"decimal_col1":null,"decimal_col2":null,"decimal_col3":null,"decimal_col4":null,"string_col":null,"binary_col":null,"date_col":null,"timestamp_col1":null,"timestamp_col2":null,"timestamp_col3":null,"char_col1":null,"char_col2":null,"char_col3":null,"varchar_col1":null,"varchar_col2":null,"varchar_col3":null,"t_map_string":null,"t_map_varchar":null,"t_map_char":null,"t_map_int":null,"t_map_bigint":null,"t_map_float":null,"t_map_double":null,"t_map_boolean":null,"t_map_decimal_precision_2":null,"t_map_decimal_precision_4":null,"t_map_decimal_precision_8":null,"t_map_decimal_precision_17":null,"t_map_decimal_precision_18":null,"t_map_decimal_precision_38":null,"t_array_string":null,"t_array_int":null,"t_array_bigint":null,"t_array_float":null,"t_array_double":null,"t_array_boolean":null,"t_array_varchar":null,"t_array_char":null,"t_array_decimal_precision_2":null,"t_array_decimal_precision_4":null,"t_array_decimal_precision_8":null,"t_array_decimal_precision_17":null,"t_array_decimal_precision_18":null,"t_array_decimal_precision_38":null,"t_struct_bigint":null,"t_complex":null,"t_struct_nested":null,"t_struct_null":null,"t_struct_non_nulls_after_nulls":null,"t_nested_struct_non_nulls_after_nulls":null,"t_map_null_value":null,"t_array_string_starting_with_nulls":null,"t_array_string_with_nulls_in_between":null,"t_array_string_ending_with_nulls":null,"t_array_string_all_nulls":null} +{"id":6,"boolean_col":null,"tinyint_col":null,"smallint_col":null,"int_col":null,"bigint_col":null,"float_col":null,"double_col":null,"decimal_col1":null,"decimal_col2":null,"decimal_col3":null,"decimal_col4":null,"string_col":null,"binary_col":null,"date_col":null,"timestamp_col1":null,"timestamp_col2":null,"timestamp_col3":null,"char_col1":null,"char_col2":null,"char_col3":null,"varchar_col1":null,"varchar_col2":null,"varchar_col3":null,"t_map_string":null,"t_map_varchar":null,"t_map_char":null,"t_map_int":null,"t_map_bigint":null,"t_map_float":null,"t_map_double":null,"t_map_boolean":null,"t_map_decimal_precision_2":null,"t_map_decimal_precision_4":null,"t_map_decimal_precision_8":null,"t_map_decimal_precision_17":null,"t_map_decimal_precision_18":null,"t_map_decimal_precision_38":null,"t_array_string":null,"t_array_int":null,"t_array_bigint":null,"t_array_float":null,"t_array_double":null,"t_array_boolean":null,"t_array_varchar":null,"t_array_char":null,"t_array_decimal_precision_2":null,"t_array_decimal_precision_4":null,"t_array_decimal_precision_8":null,"t_array_decimal_precision_17":null,"t_array_decimal_precision_18":null,"t_array_decimal_precision_38":null,"t_struct_bigint":null,"t_complex":null,"t_struct_nested":null,"t_struct_null":null,"t_struct_non_nulls_after_nulls":null,"t_nested_struct_non_nulls_after_nulls":null,"t_map_null_value":null,"t_array_string_starting_with_nulls":null,"t_array_string_with_nulls_in_between":null,"t_array_string_ending_with_nulls":null,"t_array_string_all_nulls":null} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 new file mode 100644 index 00000000000000..70d1265f98d826 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_load_data_table/1 @@ -0,0 +1,13 @@ +{"id":1,"col1":10,"col2":{"col2a":10,"col2b":"string1"},"col3":{"1":"string10"}} +{"id":2,"col1":10,"col1":20,"col2":{"col2b":"string2","col2a":0,"Col2A":20},"col3":{"2":"string2"}} +{"id":3,"col1":10,"col1":20,"COL1":30,"COL2":{"col2a":30,"col2b":"string3"}} +{"id":4,"COL1":40,"col2":{"col2a":10,"col2b":"string4","new_col":"new_val","col2a":40},"col3":{"4":"string4"}} +{"id":5} +{"id":6,"col1":60,"col2":{"COL2a":60,"col2b":600},"col3":{"6":600}} +{"id":7,"col1":70,"col3":{"7":"string7"},"col2":{"col2b":"string7","col2a":70}} + + + + +{} +{"a":5} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 new file mode 100644 index 00000000000000..11342c441bce00 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/1 @@ -0,0 +1,2 @@ +{"user_id":"user1","user_profile":{"name":"Alice","age":28,"preferences":{"sports":{"preference_id":101,"preference_values":["soccer","tennis"]},"music":{"preference_id":102,"preference_values":["rock","classical"]}}},"activity_log":[{"activity_date":"2024-08-01","activities":{"workout":{"details":"Morning run","metrics":{"duration":30.5,"calories":200.0}},"reading":{"details":"Read book on Hive","metrics":{"pages":50.0,"time":2.0}}}},{"activity_date":"2024-08-02","activities":{"travel":{"details":"Flight to NY","metrics":{"distance":500.0,"time":3.0}},"meeting":{"details":"Project meeting","metrics":{"duration":1.5,"participants":5.0}}}}]} +{"user_id":"user2","user_profile":{"name":"Bob","age":32,"preferences":{"books":{"preference_id":201,"preference_values":["fiction","non-fiction"]},"travel":{"preference_id":202,"preference_values":["beaches","mountains"]}}},"activity_log":[{"activity_date":"2024-08-01","activities":{"hiking":{"details":"Mountain trail","metrics":{"distance":10.0,"elevation":500.0}},"photography":{"details":"Wildlife photoshoot","metrics":{"photos_taken":100.0,"time":4.0}}}},{"activity_date":"2024-08-02","activities":{"workshop":{"details":"Photography workshop","metrics":{"duration":3.0,"participants":15.0}},"shopping":{"details":"Bought camera gear","metrics":{"items":5.0,"cost":1500.0}}}}]} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 new file mode 100644 index 00000000000000..e1b0befc7bca31 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/2 @@ -0,0 +1 @@ +{"user_id":"user3","user_profile":{"name":"Carol","age":24,"preferences":{"food":{"preference_id":301,"preference_values":["vegan","desserts"]},"movies":{"preference_id":302,"preference_values":["action","comedy"]}}},"activity_log":[{"activity_date":"2024-08-01","activities":{"cooking":{"details":"Made vegan meal","metrics":{"time_spent":1.5,"calories":500.0}},"movie":{"details":"Watched action movie","metrics":{"duration":2.0,"rating":8.5}}}},{"activity_date":"2024-08-02","activities":{"gym":{"details":"Strength training","metrics":{"duration":1.0,"calories":300.0}},"shopping":{"details":"Bought groceries","metrics":{"items":10.0,"cost":100.0}}}}]} diff --git a/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 new file mode 100644 index 00000000000000..08f1586f3aa91c --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/preinstalled_data/json/json_nested_complex_table/modify_2 @@ -0,0 +1,2 @@ +{"user_ID":"user4","user_PROFILE":{"name":"Carol","age":24,"preferences":{"food":{"preference_ID":301,"preference_VALUES":["vegan","desserts"]},"movies":{"preference_ID":302,"preference_VALUES":["action","comedy"]}}},"activity_LOG":[{"activity_DATE":"2024-08-01","activities":{"cooking":{"DETAILS":"Made vegan meal","metrics":{"time_spent":1.5,"calories":500.0}},"movie":{"DETAILS":"Watched action movie","metrics":{"duration":2.0,"rating":8.5}}}},{"activity_DATE":"2024-08-02","activities":{"gym":{"DETAILS":"Strength training","metrics":{"duration":1.0,"calories":300.0}},"shopping":{"DETAILS":"Bought groceries","metrics":{"items":10.0,"cost":100.0}}}}]} + diff --git a/fe/be-java-extensions/hadoop-hudi-scanner/pom.xml b/fe/be-java-extensions/hadoop-hudi-scanner/pom.xml new file mode 100644 index 00000000000000..4b80d49de17527 --- /dev/null +++ b/fe/be-java-extensions/hadoop-hudi-scanner/pom.xml @@ -0,0 +1,227 @@ + + + + + + be-java-extensions + org.apache.doris + ${revision} + + 4.0.0 + hadoop-hudi-scanner + + + ${basedir}/../../ + 1 + 0.15.0 + 1.11.3 + 1.5.4-2 + 3.1.2-22 + + + + + org.apache.doris + java-common + ${project.version} + + + org.apache.thrift + libthrift + + + + + + + org.apache.hadoop + hadoop-hdfs-client + ${hadoop.version} + + + + + org.apache.hadoop + hadoop-common + + + + org.apache.hadoop + hadoop-aws + + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + org.junit.jupiter + junit-jupiter + + + + + org.apache.hudi + hudi-common + ${hudi.version} + + + org.apache.hbase + hbase-client + + + org.apache.hbase + hbase-server + + + org.apache.thrift + libthrift + + + com.fasterxml.jackson.core + jackson-databind + + + + + + + org.apache.hudi + hudi-io + ${hudi.version} + + + + + org.apache.hudi + hudi-hadoop-mr + ${hudi.version} + + + + + org.apache.parquet + parquet-hadoop-bundle + ${parquet.version} + + + + + org.apache.parquet + parquet-avro + ${parquet.version} + + + + + org.apache.avro + avro + ${avro.version} + + + org.apache.commons + commons-compress + + + + + + io.airlift + concurrent + 202 + + + + + io.airlift + aircompressor + ${aircompressor.version} + + + + com.github.luben + zstd-jni + ${luben.zstd.jni.version} + + + + com.esotericsoftware + kryo-shaded + 4.0.2 + + + + + io.trino.hive + hive-apache + ${hive-apache.version} + + + org.apache.thrift + libthrift + + + org.apache.parquet + * + + + org.apache.avro + * + + + io.airlift + aircompressor + + + + + + + hadoop-hudi-scanner + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/main/resources/package.xml + + + + + + + + + + make-assembly + package + + single + + + + + + + diff --git a/fe/be-java-extensions/hadoop-hudi-scanner/src/main/java/org/apache/doris/hudi/HadoopHudiColumnValue.java b/fe/be-java-extensions/hadoop-hudi-scanner/src/main/java/org/apache/doris/hudi/HadoopHudiColumnValue.java new file mode 100644 index 00000000000000..ae0199d07d27c5 --- /dev/null +++ b/fe/be-java-extensions/hadoop-hudi-scanner/src/main/java/org/apache/doris/hudi/HadoopHudiColumnValue.java @@ -0,0 +1,219 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.hudi; + +import org.apache.doris.common.jni.vec.ColumnType; +import org.apache.doris.common.jni.vec.ColumnValue; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.TimestampWritableV2; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; +import org.apache.hadoop.io.LongWritable; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.Map; + +public class HadoopHudiColumnValue implements ColumnValue { + private ColumnType dorisType; + private ObjectInspector fieldInspector; + private Object fieldData; + private final ZoneId zoneId; + + public HadoopHudiColumnValue(ZoneId zoneId) { + this.zoneId = zoneId; + } + + public void setRow(Object record) { + this.fieldData = record; + } + + public void setField(ColumnType dorisType, ObjectInspector fieldInspector) { + this.dorisType = dorisType; + this.fieldInspector = fieldInspector; + } + + private Object inspectObject() { + return ((PrimitiveObjectInspector) fieldInspector).getPrimitiveJavaObject(fieldData); + } + + @Override + public boolean getBoolean() { + return (boolean) inspectObject(); + } + + @Override + public short getShort() { + return (short) inspectObject(); + } + + @Override + public int getInt() { + return (int) inspectObject(); + } + + @Override + public float getFloat() { + return (float) inspectObject(); + } + + @Override + public long getLong() { + return (long) inspectObject(); + } + + @Override + public double getDouble() { + return (double) inspectObject(); + } + + @Override + public String getString() { + return inspectObject().toString(); + } + + @Override + public byte[] getBytes() { + return (byte[]) inspectObject(); + } + + + @Override + public byte getByte() { + throw new UnsupportedOperationException("Hoodie type does not support tinyint"); + } + + @Override + public BigDecimal getDecimal() { + return ((HiveDecimal) inspectObject()).bigDecimalValue(); + } + + @Override + public LocalDate getDate() { + return LocalDate.ofEpochDay((((DateObjectInspector) fieldInspector).getPrimitiveJavaObject(fieldData)) + .toEpochDay()); + } + + @Override + public LocalDateTime getDateTime() { + if (fieldData instanceof Timestamp) { + return ((Timestamp) fieldData).toLocalDateTime(); + } else if (fieldData instanceof TimestampWritableV2) { + return LocalDateTime.ofInstant(Instant.ofEpochSecond((((TimestampObjectInspector) fieldInspector) + .getPrimitiveJavaObject(fieldData)).toEpochSecond()), zoneId); + } else { + long datetime = ((LongWritable) fieldData).get(); + long seconds; + long nanoseconds; + if (dorisType.getPrecision() == 3) { + seconds = datetime / 1000; + nanoseconds = (datetime % 1000) * 1000000; + } else if (dorisType.getPrecision() == 6) { + seconds = datetime / 1000000; + nanoseconds = (datetime % 1000000) * 1000; + } else { + throw new RuntimeException("Hoodie timestamp only support milliseconds and microseconds, " + + "wrong precision = " + dorisType.getPrecision()); + } + return LocalDateTime.ofInstant(Instant.ofEpochSecond(seconds, nanoseconds), zoneId); + } + } + + @Override + public boolean canGetStringAsBytes() { + return false; + } + + @Override + public boolean isNull() { + return fieldData == null; + } + + @Override + public BigInteger getBigInteger() { + throw new UnsupportedOperationException("Hoodie type does not support largeint"); + } + + @Override + public byte[] getStringAsBytes() { + throw new UnsupportedOperationException("Hoodie type does not support getStringAsBytes"); + } + + @Override + public void unpackArray(List values) { + ListObjectInspector inspector = (ListObjectInspector) fieldInspector; + List items = inspector.getList(fieldData); + ObjectInspector itemInspector = inspector.getListElementObjectInspector(); + for (int i = 0; i < items.size(); i++) { + Object item = items.get(i); + HadoopHudiColumnValue childValue = new HadoopHudiColumnValue(zoneId); + childValue.setRow(item); + childValue.setField(dorisType.getChildTypes().get(0), itemInspector); + values.add(childValue); + } + } + + @Override + public void unpackMap(List keys, List values) { + MapObjectInspector inspector = (MapObjectInspector) fieldInspector; + ObjectInspector keyObjectInspector = inspector.getMapKeyObjectInspector(); + ObjectInspector valueObjectInspector = inspector.getMapValueObjectInspector(); + for (Map.Entry kv : inspector.getMap(fieldData).entrySet()) { + HadoopHudiColumnValue key = new HadoopHudiColumnValue(zoneId); + key.setRow(kv.getKey()); + key.setField(dorisType.getChildTypes().get(0), keyObjectInspector); + keys.add(key); + + HadoopHudiColumnValue value = new HadoopHudiColumnValue(zoneId); + value.setRow(kv.getValue()); + value.setField(dorisType.getChildTypes().get(1), valueObjectInspector); + values.add(value); + } + } + + @Override + public void unpackStruct(List structFieldIndex, List values) { + StructObjectInspector inspector = (StructObjectInspector) fieldInspector; + List fields = inspector.getAllStructFieldRefs(); + for (int i = 0; i < structFieldIndex.size(); i++) { + Integer idx = structFieldIndex.get(i); + HadoopHudiColumnValue value = new HadoopHudiColumnValue(zoneId); + Object obj = null; + if (idx != null) { + StructField sf = fields.get(idx); + obj = inspector.getStructFieldData(fieldData, sf); + } + value.setRow(obj); + value.setField(dorisType.getChildTypes().get(i), fields.get(i).getFieldObjectInspector()); + values.add(value); + } + } +} diff --git a/fe/be-java-extensions/hadoop-hudi-scanner/src/main/java/org/apache/doris/hudi/HadoopHudiJniScanner.java b/fe/be-java-extensions/hadoop-hudi-scanner/src/main/java/org/apache/doris/hudi/HadoopHudiJniScanner.java new file mode 100644 index 00000000000000..f2b38815a366fe --- /dev/null +++ b/fe/be-java-extensions/hadoop-hudi-scanner/src/main/java/org/apache/doris/hudi/HadoopHudiJniScanner.java @@ -0,0 +1,271 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.hudi; + +import org.apache.doris.common.classloader.ThreadClassLoaderContext; +import org.apache.doris.common.jni.JniScanner; +import org.apache.doris.common.jni.vec.ColumnType; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.ZoneId; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * HadoopHudiJniScanner is a JniScanner implementation that reads Hudi data using hudi-hadoop-mr. + */ +public class HadoopHudiJniScanner extends JniScanner { + private static final Logger LOG = LoggerFactory.getLogger(HadoopHudiJniScanner.class); + + private static final String HADOOP_CONF_PREFIX = "hadoop_conf."; + + // Hudi data info + private final String basePath; + private final String dataFilePath; + private final long dataFileLength; + private final String[] deltaFilePaths; + private final String instantTime; + private final String serde; + private final String inputFormat; + + // schema info + private final String hudiColumnNames; + private final String[] hudiColumnTypes; + private final String[] requiredFields; + private List requiredColumnIds; + private ColumnType[] requiredTypes; + + // Hadoop info + private RecordReader reader; + private StructObjectInspector rowInspector; + private final ObjectInspector[] fieldInspectors; + private final StructField[] structFields; + private Deserializer deserializer; + private final Map fsOptionsProps; + + // scanner info + private final HadoopHudiColumnValue columnValue; + private final int fetchSize; + private final ClassLoader classLoader; + + public HadoopHudiJniScanner(int fetchSize, Map params) { + this.basePath = params.get("base_path"); + this.dataFilePath = params.get("data_file_path"); + this.dataFileLength = Long.parseLong(params.get("data_file_length")); + if (Strings.isNullOrEmpty(params.get("delta_file_paths"))) { + this.deltaFilePaths = new String[0]; + } else { + this.deltaFilePaths = params.get("delta_file_paths").split(","); + } + this.instantTime = params.get("instant_time"); + this.serde = params.get("serde"); + this.inputFormat = params.get("input_format"); + + this.hudiColumnNames = params.get("hudi_column_names"); + this.hudiColumnTypes = params.get("hudi_column_types").split("#"); + this.requiredFields = params.get("required_fields").split(","); + + this.fieldInspectors = new ObjectInspector[requiredFields.length]; + this.structFields = new StructField[requiredFields.length]; + this.fsOptionsProps = Maps.newHashMap(); + for (Map.Entry entry : params.entrySet()) { + if (entry.getKey().startsWith(HADOOP_CONF_PREFIX)) { + fsOptionsProps.put(entry.getKey().substring(HADOOP_CONF_PREFIX.length()), entry.getValue()); + } + if (LOG.isDebugEnabled()) { + LOG.debug("get hudi params {}: {}", entry.getKey(), entry.getValue()); + } + } + + ZoneId zoneId; + if (Strings.isNullOrEmpty(params.get("time_zone"))) { + zoneId = ZoneId.systemDefault(); + } else { + zoneId = ZoneId.of(params.get("time_zone")); + } + this.columnValue = new HadoopHudiColumnValue(zoneId); + this.fetchSize = fetchSize; + this.classLoader = this.getClass().getClassLoader(); + } + + @Override + public void open() throws IOException { + try (ThreadClassLoaderContext ignored = new ThreadClassLoaderContext(classLoader)) { + initRequiredColumnsAndTypes(); + initTableInfo(requiredTypes, requiredFields, fetchSize); + Properties properties = getReaderProperties(); + initReader(properties); + } catch (Exception e) { + close(); + LOG.warn("failed to open hadoop hudi jni scanner", e); + throw new IOException("failed to open hadoop hudi jni scanner: " + e.getMessage(), e); + } + } + + @Override + public int getNext() throws IOException { + try (ThreadClassLoaderContext ignored = new ThreadClassLoaderContext(classLoader)) { + NullWritable key = reader.createKey(); + ArrayWritable value = reader.createValue(); + int numRows = 0; + for (; numRows < fetchSize; numRows++) { + if (!reader.next(key, value)) { + break; + } + Object rowData = deserializer.deserialize(value); + for (int i = 0; i < fields.length; i++) { + Object fieldData = rowInspector.getStructFieldData(rowData, structFields[i]); + columnValue.setRow(fieldData); + // LOG.info("rows: {}, column: {}, col name: {}, col type: {}, inspector: {}", + // numRows, i, types[i].getName(), types[i].getType().name(), + // fieldInspectors[i].getTypeName()); + columnValue.setField(types[i], fieldInspectors[i]); + appendData(i, columnValue); + } + } + return numRows; + } catch (Exception e) { + close(); + LOG.warn("failed to get next in hadoop hudi jni scanner", e); + throw new IOException("failed to get next in hadoop hudi jni scanner: " + e.getMessage(), e); + } + } + + @Override + public void close() throws IOException { + try (ThreadClassLoaderContext ignored = new ThreadClassLoaderContext(classLoader)) { + if (reader != null) { + reader.close(); + } + } catch (IOException e) { + LOG.warn("failed to close hadoop hudi jni scanner", e); + throw new IOException("failed to close hadoop hudi jni scanner: " + e.getMessage(), e); + } + } + + private void initRequiredColumnsAndTypes() { + String[] splitHudiColumnNames = hudiColumnNames.split(","); + + Map hudiColNameToIdx = + IntStream.range(0, splitHudiColumnNames.length) + .boxed() + .collect(Collectors.toMap(i -> splitHudiColumnNames[i], i -> i)); + + Map hudiColNameToType = + IntStream.range(0, splitHudiColumnNames.length) + .boxed() + .collect(Collectors.toMap(i -> splitHudiColumnNames[i], i -> hudiColumnTypes[i])); + + requiredTypes = Arrays.stream(requiredFields) + .map(field -> ColumnType.parseType(field, hudiColNameToType.get(field))) + .toArray(ColumnType[]::new); + + requiredColumnIds = Arrays.stream(requiredFields) + .mapToInt(hudiColNameToIdx::get) + .boxed().collect(Collectors.toList()); + } + + private Properties getReaderProperties() { + Properties properties = new Properties(); + properties.setProperty("hive.io.file.readcolumn.ids", Joiner.on(",").join(requiredColumnIds)); + properties.setProperty("hive.io.file.readcolumn.names", Joiner.on(",").join(this.requiredFields)); + properties.setProperty("columns", this.hudiColumnNames); + properties.setProperty("columns.types", Joiner.on(",").join(hudiColumnTypes)); + properties.setProperty("serialization.lib", this.serde); + properties.setProperty("hive.io.file.read.all.columns", "false"); + fsOptionsProps.forEach(properties::setProperty); + return properties; + } + + private void initReader(Properties properties) throws Exception { + String realtimePath = dataFileLength != -1 ? dataFilePath : deltaFilePaths[0]; + long realtimeLength = dataFileLength != -1 ? dataFileLength : 0; + Path path = new Path(realtimePath); + FileSplit fileSplit = new FileSplit(path, 0, realtimeLength, (String[]) null); + List logFiles = Arrays.stream(deltaFilePaths).map(HoodieLogFile::new) + .collect(Collectors.toList()); + FileSplit hudiSplit = + new HoodieRealtimeFileSplit(fileSplit, basePath, logFiles, instantTime, false, Option.empty()); + + JobConf jobConf = new JobConf(new Configuration()); + properties.stringPropertyNames().forEach(name -> jobConf.set(name, properties.getProperty(name))); + InputFormat inputFormatClass = createInputFormat(jobConf, inputFormat); + reader = (RecordReader) inputFormatClass + .getRecordReader(hudiSplit, jobConf, Reporter.NULL); + + deserializer = getDeserializer(jobConf, properties, serde); + rowInspector = getTableObjectInspector(deserializer); + for (int i = 0; i < requiredFields.length; i++) { + StructField field = rowInspector.getStructFieldRef(requiredFields[i]); + structFields[i] = field; + fieldInspectors[i] = field.getFieldObjectInspector(); + } + } + + private InputFormat createInputFormat(Configuration conf, String inputFormat) throws Exception { + Class clazz = conf.getClassByName(inputFormat); + Class> cls = + (Class>) clazz.asSubclass(InputFormat.class); + return ReflectionUtils.newInstance(cls, conf); + } + + private Deserializer getDeserializer(Configuration configuration, Properties properties, String name) + throws Exception { + Class deserializerClass = Class.forName(name, true, JavaUtils.getClassLoader()) + .asSubclass(Deserializer.class); + Deserializer deserializer = deserializerClass.getConstructor().newInstance(); + deserializer.initialize(configuration, properties); + return deserializer; + } + + private StructObjectInspector getTableObjectInspector(Deserializer deserializer) throws Exception { + ObjectInspector inspector = deserializer.getObjectInspector(); + Preconditions.checkArgument(inspector.getCategory() == ObjectInspector.Category.STRUCT, + "expected STRUCT: %s", inspector.getCategory()); + return (StructObjectInspector) inspector; + } +} diff --git a/fe/be-java-extensions/hadoop-hudi-scanner/src/main/resources/package.xml b/fe/be-java-extensions/hadoop-hudi-scanner/src/main/resources/package.xml new file mode 100644 index 00000000000000..4bbb2610603363 --- /dev/null +++ b/fe/be-java-extensions/hadoop-hudi-scanner/src/main/resources/package.xml @@ -0,0 +1,41 @@ + + + + jar-with-dependencies + + jar + + false + + + / + true + true + runtime + + + **/Log4j2Plugins.dat + + + + + diff --git a/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java b/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java index a284c7adcdd105..bc082e56732476 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java +++ b/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java @@ -17,11 +17,10 @@ package org.apache.doris.hudi; - import org.apache.doris.common.jni.JniScanner; import org.apache.doris.common.jni.vec.ColumnType; import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopUGI; +import org.apache.doris.common.security.authentication.HadoopAuthenticator; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.avro.generic.GenericDatumReader; @@ -160,14 +159,15 @@ public void open() throws IOException { cleanResolverLock.readLock().lock(); try { lastUpdateTime.set(System.currentTimeMillis()); + AuthenticationConfig authenticationConfig = AuthenticationConfig.getKerberosConfig(split.hadoopConf()); + HadoopAuthenticator hadoopAuthenticator = HadoopAuthenticator + .getHadoopAuthenticator(authenticationConfig); if (split.incrementalRead()) { - recordIterator = HadoopUGI.ugiDoAs(AuthenticationConfig.getKerberosConfig( - split.hadoopConf()), - () -> new MORIncrementalSplitReader(split).buildScanIterator(new Filter[0])); + recordIterator = hadoopAuthenticator.doAs(() -> new MORIncrementalSplitReader(split) + .buildScanIterator(new Filter[0])); } else { - recordIterator = HadoopUGI.ugiDoAs(AuthenticationConfig.getKerberosConfig( - split.hadoopConf()), - () -> new MORSnapshotSplitReader(split).buildScanIterator(new Filter[0])); + recordIterator = hadoopAuthenticator.doAs(() -> new MORSnapshotSplitReader(split) + .buildScanIterator(new Filter[0])); } if (AVRO_RESOLVER_CACHE != null && AVRO_RESOLVER_CACHE.get() != null) { cachedResolvers.computeIfAbsent(Thread.currentThread().getId(), diff --git a/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/Utils.java b/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/Utils.java index 5614f8bcc96eb1..c0fbec633e897c 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/Utils.java +++ b/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/Utils.java @@ -18,11 +18,12 @@ package org.apache.doris.hudi; import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopUGI; +import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import java.io.BufferedReader; import java.io.File; @@ -75,7 +76,14 @@ public static void killProcess(long pid) { } public static HoodieTableMetaClient getMetaClient(Configuration conf, String basePath) { - return HadoopUGI.ugiDoAs(AuthenticationConfig.getKerberosConfig(conf), () -> HoodieTableMetaClient.builder() - .setConf(conf).setBasePath(basePath).build()); + HadoopStorageConfiguration hadoopStorageConfiguration = new HadoopStorageConfiguration(conf); + AuthenticationConfig authenticationConfig = AuthenticationConfig.getKerberosConfig(conf); + HadoopAuthenticator hadoopAuthenticator = HadoopAuthenticator.getHadoopAuthenticator(authenticationConfig); + try { + return hadoopAuthenticator.doAs(() -> HoodieTableMetaClient.builder() + .setConf(hadoopStorageConfiguration).setBasePath(basePath).build()); + } catch (IOException e) { + throw new RuntimeException("Failed to get HoodieTableMetaClient", e); + } } } diff --git a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala index dcc068ad7006d8..fc8d74f9713c26 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala +++ b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala @@ -36,13 +36,15 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, T import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.hadoop.CachingPath +import org.apache.hudi.hadoop.fs.CachingPath import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} -import org.apache.hudi.io.storage.HoodieAvroHFileReader +import org.apache.hudi.io.hadoop.HoodieHBaseAvroHFileReader import org.apache.hudi.metadata.HoodieTableMetadataUtil import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieSparkConfUtils, HoodieTableSchema, HoodieTableState} +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import org.apache.log4j.Logger import org.apache.spark.sql.adapter.Spark3_4Adapter import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters} @@ -430,7 +432,7 @@ abstract class BaseSplitReader(val split: HoodieSplit) { try { if (shouldExtractPartitionValuesFromPartitionPath) { val filePath = new Path(split.dataFilePath) - val tablePathWithoutScheme = CachingPath.getPathWithoutSchemeAndAuthority(tableInformation.metaClient.getBasePathV2) + val tablePathWithoutScheme = CachingPath.getPathWithoutSchemeAndAuthority(new Path(tableInformation.metaClient.getBasePathV2.toUri)) val partitionPathWithoutScheme = CachingPath.getPathWithoutSchemeAndAuthority(filePath.getParent) val relativePath = new URI(tablePathWithoutScheme.toString).relativize(new URI(partitionPathWithoutScheme.toString)).toString val hiveStylePartitioningEnabled = tableConfig.getHiveStylePartitioningEnable.toBoolean @@ -497,8 +499,11 @@ abstract class BaseSplitReader(val split: HoodieSplit) { options: Map[String, String], hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { partitionedFile => { - val reader = new HoodieAvroHFileReader( - hadoopConf, partitionedFile.filePath.toPath, new CacheConfig(hadoopConf)) + var hadoopStorageConfiguration = new HadoopStorageConfiguration(hadoopConf); + var storagePath = new StoragePath(partitionedFile.toPath.toUri.getPath); + var emptySchema = org.apache.hudi.common.util.Option.empty[org.apache.avro.Schema]() + val reader = new HoodieHBaseAvroHFileReader( + hadoopStorageConfiguration, storagePath, emptySchema) val requiredRowSchema = requiredDataSchema.structTypeSchema // NOTE: Schema has to be parsed at this point, since Avro's [[Schema]] aren't serializable diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java index 6cbed70adc7d46..d6325bdae4673a 100644 --- a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java +++ b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java @@ -25,6 +25,7 @@ import com.aliyun.odps.account.AliyunAccount; import com.aliyun.odps.table.configuration.CompressionCodec; import com.aliyun.odps.table.configuration.ReaderOptions; +import com.aliyun.odps.table.configuration.RestOptions; import com.aliyun.odps.table.enviroment.Credentials; import com.aliyun.odps.table.enviroment.EnvironmentSettings; import com.aliyun.odps.table.read.SplitReader; @@ -67,6 +68,10 @@ public class MaxComputeJniScanner extends JniScanner { private static final String SCAN_SERIALIZER = "scan_serializer"; private static final String TIME_ZONE = "time_zone"; + private static final String CONNECT_TIMEOUT = "connect_timeout"; + private static final String READ_TIMEOUT = "read_timeout"; + private static final String RETRY_COUNT = "retry_count"; + private enum SplitType { BYTE_SIZE, ROW_OFFSET @@ -136,16 +141,40 @@ public MaxComputeJniScanner(int batchSize, Map params) { Credentials credentials = Credentials.newBuilder().withAccount(odps.getAccount()) .withAppAccount(odps.getAppAccount()).build(); + + int connectTimeout = 10; // 10s + if (!Strings.isNullOrEmpty(params.get(CONNECT_TIMEOUT))) { + connectTimeout = Integer.parseInt(params.get(CONNECT_TIMEOUT)); + } + + int readTimeout = 120; // 120s + if (!Strings.isNullOrEmpty(params.get(READ_TIMEOUT))) { + readTimeout = Integer.parseInt(params.get(READ_TIMEOUT)); + } + + int retryTimes = 4; // 4 times + if (!Strings.isNullOrEmpty(params.get(RETRY_COUNT))) { + retryTimes = Integer.parseInt(params.get(RETRY_COUNT)); + } + + RestOptions restOptions = RestOptions.newBuilder() + .withConnectTimeout(connectTimeout) + .withReadTimeout(readTimeout) + .withRetryTimes(retryTimes).build(); + settings = EnvironmentSettings.newBuilder() .withCredentials(credentials) .withServiceEndpoint(odps.getEndpoint()) .withQuotaName(quota) + .withRestOptions(restOptions) .build(); try { scan = (TableBatchReadSession) deserialize(scanSerializer); } catch (Exception e) { - LOG.info("deserialize TableBatchReadSession failed.", e); + String errorMsg = "Failed to deserialize table batch read session."; + LOG.warn(errorMsg, e); + throw new IllegalArgumentException(errorMsg, e); } } @@ -176,11 +205,11 @@ public void open() throws IOException { .withReuseBatch(true) .build()); - } catch (IOException e) { - LOG.info("createArrowReader failed.", e); } catch (Exception e) { + String errorMsg = "MaxComputeJniScanner Failed to open table batch read session."; + LOG.warn(errorMsg, e); close(); - throw new IOException(e.getMessage(), e); + throw new IOException(errorMsg, e); } } @@ -215,8 +244,9 @@ private int readVectors(int expectedRows) throws IOException { break; } } catch (Exception e) { - LOG.info("currentSplitReader hasNext fail", e); - break; + String errorMsg = "MaxComputeJniScanner readVectors hasNext fail"; + LOG.warn(errorMsg, e); + throw new IOException(e.getMessage(), e); } try { @@ -241,7 +271,10 @@ private int readVectors(int expectedRows) throws IOException { } curReadRows += batchRows; } catch (Exception e) { - throw new RuntimeException("Fail to read arrow data, reason: " + e.getMessage(), e); + String errorMsg = String.format("MaxComputeJniScanner Fail to read arrow data. " + + "curReadRows = {}, expectedRows = {}", curReadRows, expectedRows); + LOG.warn(errorMsg, e); + throw new RuntimeException(errorMsg, e); } } return curReadRows; diff --git a/fe/be-java-extensions/pom.xml b/fe/be-java-extensions/pom.xml index bbe056739d51ec..5d56ef76e7c3ef 100644 --- a/fe/be-java-extensions/pom.xml +++ b/fe/be-java-extensions/pom.xml @@ -22,6 +22,7 @@ under the License. 4.0.0 hudi-scanner + hadoop-hudi-scanner java-common java-udf jdbc-scanner 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 568cd7c817e756..fa98487d19f745 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 @@ -2903,7 +2903,7 @@ public class Config extends ConfigBase { "Columns that have not been collected within the specified interval will trigger automatic analyze. " + "0 means not trigger." }) - public static long auto_analyze_interval_seconds = 0; + public static long auto_analyze_interval_seconds = 86400; // 24 hours. //========================================================================== // begin of cloud config diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopUGI.java b/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopUGI.java deleted file mode 100644 index 2f73440ecfa368..00000000000000 --- a/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopUGI.java +++ /dev/null @@ -1,103 +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. - -package org.apache.doris.common.security.authentication; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.io.IOException; -import java.security.PrivilegedExceptionAction; - -@Deprecated -public class HadoopUGI { - private static final Logger LOG = LogManager.getLogger(HadoopUGI.class); - - /** - * login and return hadoop ugi - * @param config auth config - * @return ugi - */ - private static UserGroupInformation loginWithUGI(AuthenticationConfig config) { - if (config == null || !config.isValid()) { - return null; - } - if (config instanceof KerberosAuthenticationConfig) { - try { - // TODO: remove after iceberg and hudi kerberos test case pass - try { - // login hadoop with keytab and try checking TGT - UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); - LOG.debug("Current login user: {}", ugi.getUserName()); - String principal = ((KerberosAuthenticationConfig) config).getKerberosPrincipal(); - if (ugi.hasKerberosCredentials() && StringUtils.equals(ugi.getUserName(), principal)) { - // if the current user is logged by kerberos and is the same user - // just use checkTGTAndReloginFromKeytab because this method will only relogin - // when the TGT is expired or is close to expiry - ugi.checkTGTAndReloginFromKeytab(); - return ugi; - } - } catch (IOException e) { - LOG.warn("A SecurityException occurs with kerberos, do login immediately.", e); - } - return new HadoopKerberosAuthenticator((KerberosAuthenticationConfig) config).getUGI(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } else { - String hadoopUserName = ((SimpleAuthenticationConfig) config).getUsername(); - if (hadoopUserName == null) { - hadoopUserName = "hadoop"; - ((SimpleAuthenticationConfig) config).setUsername(hadoopUserName); - LOG.debug(AuthenticationConfig.HADOOP_USER_NAME + " is unset, use default user: hadoop"); - } - - UserGroupInformation ugi; - try { - ugi = UserGroupInformation.getLoginUser(); - if (ugi.getUserName().equals(hadoopUserName)) { - return ugi; - } - } catch (IOException e) { - LOG.warn("A SecurityException occurs with simple, do login immediately.", e); - } - - ugi = UserGroupInformation.createRemoteUser(hadoopUserName); - UserGroupInformation.setLoginUser(ugi); - LOG.debug("Login by proxy user, hadoop.username: {}", hadoopUserName); - return ugi; - } - } - - public static T ugiDoAs(AuthenticationConfig authConf, PrivilegedExceptionAction action) { - UserGroupInformation ugi = HadoopUGI.loginWithUGI(authConf); - try { - if (ugi != null) { - if (authConf instanceof KerberosAuthenticationConfig) { - ugi.checkTGTAndReloginFromKeytab(); - } - return ugi.doAs(action); - } else { - return action.run(); - } - } catch (Exception e) { - throw new RuntimeException(e.getMessage(), e); - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java index 2715bd1f6da2f9..47fdfdce4e23c8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java @@ -88,6 +88,9 @@ public void analyze(Analyzer analyzer) throws UserException { public String toSql() { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP DATABASE ").append("`").append(dbName).append("`"); + if (forceDrop) { + stringBuilder.append(" FORCE"); + } return stringBuilder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java index 5e06fce75ee074..d6a19e81f8e3f4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java @@ -100,6 +100,9 @@ public void analyze(Analyzer analyzer) throws UserException { public String toSql() { StringBuilder stringBuilder = new StringBuilder(); stringBuilder.append("DROP TABLE ").append(tableName.toSql()); + if (forceDrop) { + stringBuilder.append(" FORCE"); + } return stringBuilder.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java index f9bb218744fc85..5f9872e42ad7cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropUserStmt.java @@ -63,8 +63,8 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { userIdent.analyze(); - if (userIdent.isRootUser()) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop root user"); + if (userIdent.isSystemUser()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Can not drop system user"); } // only user with GLOBAL level's GRANT_PRIV can drop user. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java index 7f406d2a7e1365..f51e63e4fbec0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java @@ -43,12 +43,17 @@ public class IndexDef { private boolean isBuildDeferred = false; private PartitionNames partitionNames; private List columnUniqueIds = Lists.newArrayList(); + private static final int MIN_NGRAM_SIZE = 1; + private static final int MAX_NGRAM_SIZE = 255; + private static final int MIN_BF_SIZE = 64; + private static final int MAX_BF_SIZE = 65535; public static final String NGRAM_SIZE_KEY = "gram_size"; public static final String NGRAM_BF_SIZE_KEY = "bf_size"; public static final String DEFAULT_NGRAM_SIZE = "2"; public static final String DEFAULT_NGRAM_BF_SIZE = "256"; + public IndexDef(String indexName, boolean ifNotExists, List columns, IndexType indexType, Map properties, String comment) { this.indexName = indexName; @@ -238,8 +243,8 @@ public void checkColumn(Column column, KeysType keysType, boolean enableUniqueKe throw new AnalysisException("index should only be used in columns of DUP_KEYS/UNIQUE_KEYS table" + " or key columns of AGG_KEYS table. invalid index: " + indexName); } else if (keysType == KeysType.UNIQUE_KEYS && !enableUniqueKeyMergeOnWrite - && indexType == IndexType.INVERTED && properties != null - && properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)) { + && indexType == IndexType.INVERTED && properties != null + && properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)) { throw new AnalysisException("INVERTED index with parser can NOT be used in value columns of" + " UNIQUE_KEYS table with merge_on_write disable. invalid index: " + indexName); } @@ -256,21 +261,29 @@ public void checkColumn(Column column, KeysType keysType, boolean enableUniqueKe if (properties.size() != 2) { throw new AnalysisException("ngram_bf index should have gram_size and bf_size properties"); } - try { - int ngramSize = Integer.parseInt(properties.get(NGRAM_SIZE_KEY)); - int bfSize = Integer.parseInt(properties.get(NGRAM_BF_SIZE_KEY)); - if (ngramSize > 256 || ngramSize < 1) { - throw new AnalysisException("gram_size should be integer and less than 256"); - } - if (bfSize > 65535 || bfSize < 64) { - throw new AnalysisException("bf_size should be integer and between 64 and 65535"); - } - } catch (NumberFormatException e) { - throw new AnalysisException("invalid ngram properties:" + e.getMessage(), e); - } + + parseAndValidateProperty(properties, NGRAM_SIZE_KEY, MIN_NGRAM_SIZE, MAX_NGRAM_SIZE); + parseAndValidateProperty(properties, NGRAM_BF_SIZE_KEY, MIN_BF_SIZE, MAX_BF_SIZE); } } else { throw new AnalysisException("Unsupported index type: " + indexType); } } + + private void parseAndValidateProperty(Map properties, String key, int minValue, int maxValue) + throws AnalysisException { + String valueStr = properties.get(key); + if (valueStr == null) { + throw new AnalysisException("Property '" + key + "' is missing."); + } + try { + int value = Integer.parseInt(valueStr); + if (value < minValue || value > maxValue) { + throw new AnalysisException("'" + key + "' should be an integer between " + + minValue + " and " + maxValue + "."); + } + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid value for '" + key + "': " + valueStr, e); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java index ef65b405853765..026e4da29b59da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java @@ -36,7 +36,6 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.property.PropertyConverter; import org.apache.doris.datasource.property.constants.S3Properties; -import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TParquetCompressionType; @@ -302,11 +301,8 @@ private String dorisTypeToOrcTypeMap(Type dorisType) throws AnalysisException { break; case HLL: case BITMAP: - if (!(ConnectContext.get() != null && ConnectContext.get() - .getSessionVariable().isReturnObjectDataAsBinary())) { - break; - } - orcType = "string"; + case QUANTILE_STATE: + orcType = "binary"; break; case DATEV2: orcType = "date"; @@ -327,6 +323,8 @@ private String dorisTypeToOrcTypeMap(Type dorisType) throws AnalysisException { case DATE: case DATETIME: case IPV6: + case VARIANT: + case JSONB: orcType = "string"; break; case DECIMALV2: @@ -445,6 +443,8 @@ private void analyzeForOrcFormat(List resultExprs, List colLabels) case DATE: case DATETIME: case IPV6: + case VARIANT: + case JSONB: checkOrcType(schema.second, "string", true, resultType.getPrimitiveType().toString()); break; case DECIMAL32: @@ -455,13 +455,8 @@ private void analyzeForOrcFormat(List resultExprs, List colLabels) break; case HLL: case BITMAP: - if (ConnectContext.get() != null && ConnectContext.get() - .getSessionVariable().isReturnObjectDataAsBinary()) { - checkOrcType(schema.second, "string", true, resultType.getPrimitiveType().toString()); - } else { - throw new AnalysisException("Orc format does not support column type: " - + resultType.getPrimitiveType()); - } + case QUANTILE_STATE: + checkOrcType(schema.second, "binary", true, resultType.getPrimitiveType().toString()); break; case STRUCT: checkOrcType(schema.second, "struct", false, resultType.getPrimitiveType().toString()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java index 9b07796df784d4..7cff1e2b949455 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAutoAnalyzeJobsStmt.java @@ -40,7 +40,7 @@ * [TABLE] * [ * WHERE - * [PRIORITY = ["HIGH"|"MID"|"LOW"]] + * [PRIORITY = ["HIGH"|"MID"|"LOW"|"VERY_LOW"]] * ] */ public class ShowAutoAnalyzeJobsStmt extends ShowStmt implements NotFallbackInParser { @@ -175,7 +175,7 @@ private void analyzeSubPredicate(Expr subExpr) throws AnalysisException { if (!valid) { throw new AnalysisException("Where clause should looks like: " - + "PRIORITY = \"HIGH|MID|LOW\""); + + "PRIORITY = \"HIGH|MID|LOW|VERY_LOW\""); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java index 354b57bc55c7d9..d180ef0d807fff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowColumnStatsStmt.java @@ -70,6 +70,7 @@ public class ShowColumnStatsStmt extends ShowStmt implements NotFallbackInParser .add("updated_time") .add("update_rows") .add("last_analyze_row_count") + .add("last_analyze_version") .build(); private static final ImmutableList PARTITION_COLUMN_TITLE_NAMES = @@ -185,6 +186,7 @@ public ShowResultSet constructResultSet(List, ColumnSt row.add(String.valueOf(p.second.updatedTime)); row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.updatedRows)); row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.rowCount)); + row.add(String.valueOf(colStatsMeta == null ? "N/A" : colStatsMeta.tableVersion)); result.add(row); }); return new ShowResultSet(getMetaData(), result); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java index ccffee3086dbea..ea9b96d0afecd4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTableStatsStmt.java @@ -62,6 +62,7 @@ public class ShowTableStatsStmt extends ShowStmt implements NotFallbackInParser .add("new_partition") .add("user_inject") .add("enable_auto_analyze") + .add("last_analyze_time") .build(); private static final ImmutableList PARTITION_TITLE_NAMES = @@ -230,6 +231,7 @@ public ShowResultSet constructTableResultSet(TableStatsMeta tableStatistic, Tabl row.add(""); row.add(""); row.add(String.valueOf(table.autoAnalyzeEnabled())); + row.add(""); result.add(row); return new ShowResultSet(getMetaData(), result); } @@ -242,13 +244,16 @@ public ShowResultSet constructTableResultSet(TableStatsMeta tableStatistic, Tabl LocalDateTime dateTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(tableStatistic.updatedTime), java.time.ZoneId.systemDefault()); - String formattedDateTime = dateTime.format(formatter); - row.add(formattedDateTime); + LocalDateTime lastAnalyzeTime = + LocalDateTime.ofInstant(Instant.ofEpochMilli(tableStatistic.lastAnalyzeTime), + java.time.ZoneId.systemDefault()); + row.add(dateTime.format(formatter)); row.add(tableStatistic.analyzeColumns().toString()); row.add(tableStatistic.jobType.toString()); row.add(String.valueOf(tableStatistic.partitionChanged.get())); row.add(String.valueOf(tableStatistic.userInjected)); row.add(table == null ? "N/A" : String.valueOf(table.autoAnalyzeEnabled())); + row.add(lastAnalyzeTime.format(formatter)); result.add(row); return new ShowResultSet(getMetaData(), result); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java index 122f28ca45092b..6898915dab73ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UserIdentity.java @@ -189,6 +189,10 @@ public boolean isAdminUser() { return user.equals(Auth.ADMIN_USER); } + public boolean isSystemUser() { + return isRootUser() || isAdminUser(); + } + public TUserIdentity toThrift() { Preconditions.checkState(isAnalyzed); TUserIdentity tUserIdent = new TUserIdentity(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java index 8c7eaabe51ded6..624e701c03318c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java @@ -56,6 +56,8 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum; import org.apache.doris.nereids.trees.expressions.functions.agg.MultiDistinctSum0; import org.apache.doris.nereids.trees.expressions.functions.agg.Ndv; +import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapExprCalculate; +import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapExprCalculateCount; import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapIntersect; import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapIntersectCount; import org.apache.doris.nereids.trees.expressions.functions.agg.OrthogonalBitmapUnionCount; @@ -121,6 +123,7 @@ public class BuiltinAggregateFunctions implements FunctionHelper { agg(HllUnion.class, "hll_raw_agg", "hll_union"), agg(HllUnionAgg.class, "hll_union_agg"), agg(IntersectCount.class, "intersect_count"), + agg(Kurt.class, "kurt", "kurt_pop", "kurtosis"), agg(MapAgg.class, "map_agg"), agg(Max.class, "max"), agg(MaxBy.class, "max_by"), @@ -131,17 +134,20 @@ public class BuiltinAggregateFunctions implements FunctionHelper { agg(MultiDistinctSum.class, "multi_distinct_sum"), agg(MultiDistinctSum0.class, "multi_distinct_sum0"), agg(Ndv.class, "approx_count_distinct", "ndv"), + agg(OrthogonalBitmapExprCalculate.class, "orthogonal_bitmap_expr_calculate"), + agg(OrthogonalBitmapExprCalculateCount.class, "orthogonal_bitmap_expr_calculate_count"), agg(OrthogonalBitmapIntersect.class, "orthogonal_bitmap_intersect"), agg(OrthogonalBitmapIntersectCount.class, "orthogonal_bitmap_intersect_count"), - agg(OrthogonalBitmapUnionCount.class, "orthogonal_bitmap_union_count"), - agg(Percentile.class, "percentile"), - agg(PercentileApprox.class, "percentile_approx"), - agg(PercentileApproxWeighted.class, "percentile_approx_weighted"), - agg(PercentileArray.class, "percentile_array"), - agg(QuantileUnion.class, "quantile_union"), - agg(Retention.class, "retention"), + agg(OrthogonalBitmapUnionCount.class, "orthogonal_bitmap_union_count"), + agg(Percentile.class, "percentile"), + agg(PercentileApprox.class, "percentile_approx"), + agg(PercentileApproxWeighted.class, "percentile_approx_weighted"), + agg(PercentileArray.class, "percentile_array"), + agg(QuantileUnion.class, "quantile_union"), + agg(Retention.class, "retention"), agg(SequenceCount.class, "sequence_count"), agg(SequenceMatch.class, "sequence_match"), + agg(Skew.class, "skew", "skew_pop", "skewness"), agg(Stddev.class, "stddev_pop", "stddev"), agg(StddevSamp.class, "stddev_samp"), agg(Sum.class, "sum"), @@ -151,9 +157,7 @@ public class BuiltinAggregateFunctions implements FunctionHelper { agg(TopNWeighted.class, "topn_weighted"), agg(Variance.class, "var_pop", "variance_pop", "variance"), agg(VarianceSamp.class, "var_samp", "variance_samp"), - agg(WindowFunnel.class, "window_funnel"), - agg(Skew.class, "skew", "skew_pop", "skewness"), - agg(Kurt.class, "kurt", "kurt_pop", "kurtosis") + agg(WindowFunnel.class, "window_funnel") ); public final Set aggFuncNames = aggregateFunctions.stream() 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 b0eaa75e3ca29f..44220dda10683e 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 @@ -277,6 +277,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Lower; 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.LtrimIn; 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; @@ -356,6 +357,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.RoundBankers; import org.apache.doris.nereids.trees.expressions.functions.scalar.Rpad; import org.apache.doris.nereids.trees.expressions.functions.scalar.Rtrim; +import org.apache.doris.nereids.trees.expressions.functions.scalar.RtrimIn; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecToTime; import org.apache.doris.nereids.trees.expressions.functions.scalar.Second; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondCeil; @@ -435,6 +437,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize; import org.apache.doris.nereids.trees.expressions.functions.scalar.Translate; import org.apache.doris.nereids.trees.expressions.functions.scalar.Trim; +import org.apache.doris.nereids.trees.expressions.functions.scalar.TrimIn; import org.apache.doris.nereids.trees.expressions.functions.scalar.Truncate; import org.apache.doris.nereids.trees.expressions.functions.scalar.Unhex; import org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp; @@ -755,6 +758,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(Lower.class, "lcase", "lower"), scalar(Lpad.class, "lpad"), scalar(Ltrim.class, "ltrim"), + scalar(LtrimIn.class, "ltrim_in"), scalar(MakeDate.class, "makedate"), scalar(MapContainsKey.class, "map_contains_key"), scalar(MapContainsValue.class, "map_contains_value"), @@ -830,6 +834,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(RoundBankers.class, "round_bankers"), scalar(Rpad.class, "rpad"), scalar(Rtrim.class, "rtrim"), + scalar(RtrimIn.class, "rtrim_in"), scalar(Second.class, "second"), scalar(SecondCeil.class, "second_ceil"), scalar(SecondFloor.class, "second_floor"), @@ -914,6 +919,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(ToQuantileState.class, "to_quantile_state"), scalar(Translate.class, "translate"), scalar(Trim.class, "trim"), + scalar(TrimIn.class, "trim_in"), scalar(Truncate.class, "truncate"), scalar(Unhex.class, "unhex"), scalar(UnixTimestamp.class, "unix_timestamp"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index a5505133ca2852..6063ff89979734 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -448,6 +448,20 @@ public boolean isVersionColumn() { || aggregationType == AggregateType.NONE) && nameEquals(VERSION_COL, true); } + // now we only support BloomFilter on (same behavior with BE): + // smallint/int/bigint/largeint + // string/varchar/char/variant + // date/datetime/datev2/datetimev2 + // decimal/decimal32/decimal64/decimal128I/decimal256 + // ipv4/ipv6 + public boolean isSupportBloomFilter() { + PrimitiveType pType = getDataType(); + return (pType == PrimitiveType.SMALLINT || pType == PrimitiveType.INT + || pType == PrimitiveType.BIGINT || pType == PrimitiveType.LARGEINT) + || pType.isCharFamily() || pType.isDateType() || pType.isVariantType() + || pType.isDecimalV2Type() || pType.isDecimalV3Type() || pType.isIPType(); + } + public PrimitiveType getDataType() { return type.getPrimitiveType(); } @@ -994,7 +1008,7 @@ public String toSql(boolean isUniqueTable, boolean isCompatible) { sb.append(" ON UPDATE ").append(defaultValue).append(""); } if (StringUtils.isNotBlank(comment)) { - sb.append(" COMMENT '").append(getComment(true)).append("'"); + sb.append(" COMMENT \"").append(getComment(true)).append("\""); } return sb.toString(); } 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 61f44a3526b09e..38dcd3e9a291bd 100644 --- 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 @@ -3414,25 +3414,33 @@ public static void getSyncedDdlStmt(TableIf table, List createTableStmt, hidePassword, false, specificVersion, false, true); } - public static String getMTMVDdl(MTMV mtmv) { - StringBuilder sb = new StringBuilder("CREATE MATERIALIZED VIEW "); - sb.append(mtmv.getName()); - addMTMVCols(mtmv, sb); - sb.append("\n"); - sb.append(mtmv.getRefreshInfo()); - addMTMVKeyInfo(mtmv, sb); - addTableComment(mtmv, sb); - addMTMVPartitionInfo(mtmv, sb); - DistributionInfo distributionInfo = mtmv.getDefaultDistributionInfo(); - sb.append("\n").append(distributionInfo.toSql()); - // properties - sb.append("\nPROPERTIES (\n"); - addOlapTablePropertyInfo(mtmv, sb, false, false, null); - addMTMVPropertyInfo(mtmv, sb); - sb.append("\n)"); - sb.append("\nAS "); - sb.append(mtmv.getQuerySql()); - return sb.toString(); + public static String getMTMVDdl(MTMV mtmv) throws AnalysisException { + if (!mtmv.tryReadLock(1, TimeUnit.MINUTES)) { + throw new AnalysisException( + "get table read lock timeout, database=" + mtmv.getDBName() + ",table=" + mtmv.getName()); + } + try { + StringBuilder sb = new StringBuilder("CREATE MATERIALIZED VIEW "); + sb.append(mtmv.getName()); + addMTMVCols(mtmv, sb); + sb.append("\n"); + sb.append(mtmv.getRefreshInfo()); + addMTMVKeyInfo(mtmv, sb); + addTableComment(mtmv, sb); + addMTMVPartitionInfo(mtmv, sb); + DistributionInfo distributionInfo = mtmv.getDefaultDistributionInfo(); + sb.append("\n").append(distributionInfo.toSql()); + // properties + sb.append("\nPROPERTIES (\n"); + addOlapTablePropertyInfo(mtmv, sb, false, false, null); + addMTMVPropertyInfo(mtmv, sb); + sb.append("\n)"); + sb.append("\nAS "); + sb.append(mtmv.getQuerySql()); + return sb.toString(); + } finally { + mtmv.readUnlock(); + } } private static void addMTMVKeyInfo(MTMV mtmv, StringBuilder sb) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index 955bfd4279fd5c..daf1aac333d653 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -392,7 +392,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(); + Map mvPartitionItems = getAndCopyPartitionItemsWithoutLock(); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); 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 699d62397ac834..de0adc82d1782d 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 @@ -874,17 +874,6 @@ public Set getSchemaAllIndexes(boolean full) { return columns; } - public List getMvColumns(boolean full) { - List columns = Lists.newArrayList(); - for (Long indexId : indexIdToMeta.keySet()) { - if (indexId == baseIndexId) { - continue; - } - columns.addAll(getSchemaByIndexId(indexId, full)); - } - return columns; - } - public List getBaseSchemaKeyColumns() { return getKeyColumnsByIndexId(baseIndexId); } @@ -2713,6 +2702,9 @@ public boolean getEnableUniqueKeyMergeOnWrite() { if (tableProperty == null) { return false; } + if (getKeysType() != KeysType.UNIQUE_KEYS) { + return false; + } return tableProperty.getEnableUniqueKeyMergeOnWrite(); } @@ -3152,19 +3144,23 @@ public Map getAndCopyPartitionItems() throws AnalysisExce throw new AnalysisException("get table read lock timeout, database=" + getDBName() + ",table=" + getName()); } try { - Map res = Maps.newHashMap(); - for (Entry entry : getPartitionInfo().getIdToItem(false).entrySet()) { - Partition partition = idToPartition.get(entry.getKey()); - if (partition != null) { - res.put(partition.getName(), entry.getValue()); - } - } - return res; + return getAndCopyPartitionItemsWithoutLock(); } finally { readUnlock(); } } + public Map getAndCopyPartitionItemsWithoutLock() throws AnalysisException { + Map res = Maps.newHashMap(); + for (Entry entry : getPartitionInfo().getIdToItem(false).entrySet()) { + Partition partition = idToPartition.get(entry.getKey()); + if (partition != null) { + res.put(partition.getName(), entry.getValue()); + } + } + return res; + } + @Override public List getPartitionColumns(Optional snapshot) { return getPartitionColumns(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java index f9640ccae14023..29bfda8b201d20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java @@ -373,15 +373,14 @@ public String toSql() { if (expr == MaxLiteral.MAX_VALUE || expr.isNullLiteral()) { value = expr.toSql(); sb.append(value); - continue; } else { value = "\"" + expr.getRealValue() + "\""; if (expr instanceof DateLiteral) { DateLiteral dateLiteral = (DateLiteral) expr; value = dateLiteral.toSql(); } + sb.append(value); } - sb.append(value); if (keys.size() - 1 != i) { sb.append(", "); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java index e27339c2aacc14..9468c8acecd032 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudClusterChecker.java @@ -170,6 +170,10 @@ private void updateStatus(List currentBes, List expec String endpoint = addr + ":" + node.getHeartbeatPort(); Cloud.NodeStatusPB status = node.getStatus(); Backend be = currentMap.get(endpoint); + if (be == null) { + LOG.warn("cant get valid be {} from fe mem, ignore it checker will add this be at next", endpoint); + continue; + } if (status == Cloud.NodeStatusPB.NODE_STATUS_DECOMMISSIONING) { if (!be.isDecommissioned()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java index f4c6005a0d828a..89338c228fc0b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnv.java @@ -415,7 +415,8 @@ public void dropFrontend(FrontendNodeType role, String host, int port) throws Dd Frontend frontend = checkFeExist(host, port); if (frontend == null) { - throw new DdlException("Frontend does not exist."); + throw new DdlException("frontend does not exist[" + NetUtils + .getHostPortInAccessibleFormat(host, port) + "]"); } if (frontend.getRole() != role) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java index eb75bc9312aab4..ede8cb56258636 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java @@ -126,7 +126,9 @@ public static void getFrontendsInfo(Env env, List> infos) { selfNode = ConnectContext.get().getCurrentConnectedFEIp(); } - for (Frontend fe : env.getFrontends(null /* all */)) { + List envFes = env.getFrontends(null /* all */); + LOG.info("bdbje fes {}, env fes {}", allFe, envFes); + for (Frontend fe : envFes) { List info = new ArrayList(); info.add(fe.getNodeName()); info.add(fe.getHost()); @@ -211,11 +213,6 @@ private static boolean isJoin(List allFeHosts, Frontend fe) { if (fe.getEditLogPort() != addr.getPort()) { continue; } - if (!Strings.isNullOrEmpty(addr.getHostName())) { - if (addr.getHostName().equals(fe.getHost())) { - return true; - } - } // if hostname of InetSocketAddress is ip, addr.getHostName() may be not equal to fe.getIp() // so we need to compare fe.getIp() with address.getHostAddress() InetAddress address = addr.getAddress(); @@ -227,6 +224,22 @@ private static boolean isJoin(List allFeHosts, Frontend fe) { return true; } } + + // Avoid calling getHostName multiple times, don't remove it + for (InetSocketAddress addr : allFeHosts) { + // Avoid calling getHostName multiple times, don't remove it + if (fe.getEditLogPort() != addr.getPort()) { + continue; + } + // https://bugs.openjdk.org/browse/JDK-8143378#:~:text=getHostName()%3B%20takes%20about%205,millisecond%20on%20JDK%20update%2051 + // getHostName sometime has bug, take 5s + String host = addr.getHostName(); + if (!Strings.isNullOrEmpty(host)) { + if (host.equals(fe.getHost())) { + return true; + } + } + } return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java index 9e91eb88148d3c..6c36a926d088b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletHealthProcDir.java @@ -67,7 +67,7 @@ public class TabletHealthProcDir implements ProcDirInterface { private Env env; - private ForkJoinPool taskPool = new ForkJoinPool(); + private static ForkJoinPool taskPool = new ForkJoinPool(); public TabletHealthProcDir(Env env) { Preconditions.checkNotNull(env); 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 a34827303ec055..e3af87989b8b39 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 @@ -634,6 +634,9 @@ public static Set analyzeBloomFilterColumns(Map properti } String[] bfColumnArr = bfColumnsStr.split(COMMA_SEPARATOR); + if (bfColumnArr.length == 0) { + return bfColumns; + } Set bfColumnSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); for (String bfColumn : bfColumnArr) { bfColumn = bfColumn.trim(); @@ -642,11 +645,8 @@ public static Set analyzeBloomFilterColumns(Map properti if (column.getName().equalsIgnoreCase(bfColumn)) { PrimitiveType type = column.getDataType(); - // tinyint/float/double columns don't support // key columns and none/replace aggregate non-key columns support - if (type == PrimitiveType.TINYINT || type == PrimitiveType.FLOAT - || type == PrimitiveType.DOUBLE || type == PrimitiveType.BOOLEAN - || type.isComplexType()) { + if (!column.isSupportBloomFilter()) { throw new AnalysisException(type + " is not supported in bloom filter index. " + "invalid column: " + bfColumn); } else if (keysType != KeysType.AGG_KEYS || column.isKey()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/RuntimeProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/RuntimeProfile.java index 3ffc303a6db89d..19082959034156 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/RuntimeProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/RuntimeProfile.java @@ -682,8 +682,17 @@ public void addChildWithCheck(RuntimeProfile child, Map planNod if (planNodeMap == null || !planNodeMap.containsKey(child.nodeId())) { return; } - child.addPlanNodeInfos(planNodeMap.get(child.nodeId())); - planNodeMap.remove(child.nodeId()); + + /* + * The check for SINK_OPERATOR is performed because SINK_OPERATOR does not have + * a corresponding plan node ID. + * Currently, the plan node info is only printed for non-sink nodes in the merge + * profile. + */ + if (name.contains("_SINK_OPERATOR")) { + child.addPlanNodeInfos(planNodeMap.get(child.nodeId())); + planNodeMap.remove(child.nodeId()); + } } public void addPlanNodeInfos(String infos) { 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 23e6d68954edfb..cde08113373aee 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 @@ -39,6 +39,7 @@ import org.apache.doris.common.Version; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.es.EsExternalDatabase; import org.apache.doris.datasource.hive.HMSExternalCatalog; @@ -149,6 +150,7 @@ public abstract class ExternalCatalog protected Optional useMetaCache = Optional.empty(); protected MetaCache> metaCache; + protected PreExecutionAuthenticator preExecutionAuthenticator; public ExternalCatalog() { } @@ -935,4 +937,8 @@ public void setAutoAnalyzePolicy(String dbName, String tableName, String policy) tableAutoAnalyzePolicy.put(key, policy); } } + + public PreExecutionAuthenticator getPreExecutionAuthenticator() { + return preExecutionAuthenticator; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 93710f0a129911..bd1e36e7bc968b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -382,7 +382,7 @@ public TableIndexes getTableIndexes() { * @return */ public SelectedPartitions initSelectedPartitions(Optional snapshot) { - if (!supportPartitionPruned()) { + if (!supportInternalPartitionPruned()) { return SelectedPartitions.NOT_PRUNED; } if (CollectionUtils.isEmpty(this.getPartitionColumns(snapshot))) { @@ -399,7 +399,7 @@ public SelectedPartitions initSelectedPartitions(Optional snapshot * @param snapshot if not support mvcc, ignore this * @return partitionName ==> PartitionItem */ - public Map getNameToPartitionItems(Optional snapshot) { + protected Map getNameToPartitionItems(Optional snapshot) { return Collections.emptyMap(); } @@ -415,11 +415,12 @@ public List getPartitionColumns(Optional snapshot) { } /** - * Does it support partition cpruned, If so, this method needs to be overridden in subclasses + * Does it support Internal partition pruned, If so, this method needs to be overridden in subclasses + * Internal partition pruned : Implement partition pruning logic without relying on external APIs. * * @return */ - public boolean supportPartitionPruned() { + public boolean supportInternalPartitionPruned() { return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index d1de1306d04693..5ea2a2637d86e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -433,6 +433,8 @@ private TScanRangeLocations splitToScanRange( } } + // set file format type, and the type might fall back to native format in setScanParams + rangeDesc.setFormatType(getFileFormatType()); setScanParams(rangeDesc, fileSplit); curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); TScanRangeLocation location = new TScanRangeLocation(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java index 9845c0070fc430..29fdb2b09acfcd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileScanNode.java @@ -126,7 +126,6 @@ public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { output.append(prefix); if (isBatchMode()) { output.append("(approximate)"); - splitAssignment.stop(); } output.append("inputSplitNum=").append(selectedSplitNum).append(", totalFileSize=") .append(totalFileSize).append(", scanRanges=").append(scanRangeLocations.size()).append("\n"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java index 20b9482041df02..85b999f1111047 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalCatalog.java @@ -27,6 +27,7 @@ import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.security.authentication.AuthenticationConfig; import org.apache.doris.common.security.authentication.HadoopAuthenticator; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; @@ -34,6 +35,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.SessionContext; +import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; import org.apache.doris.datasource.operations.ExternalMetadataOperations; @@ -88,7 +90,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private boolean enableHmsEventsIncrementalSync = false; //for "type" = "hms" , but is iceberg table. - private HiveCatalog icebergHiveCatalog; + private IcebergMetadataOps icebergMetadataOps; @VisibleForTesting public HMSExternalCatalog() { @@ -168,6 +170,7 @@ public void checkProperties() throws DdlException { @Override protected void initLocalObjectsImpl() { + preExecutionAuthenticator = new PreExecutionAuthenticator(); if (authenticator == null) { AuthenticationConfig config = AuthenticationConfig.getKerberosConfig(getConfiguration()); authenticator = HadoopAuthenticator.getHadoopAuthenticator(config); @@ -199,8 +202,6 @@ protected void initLocalObjectsImpl() { transactionManager = TransactionManagerFactory.createHiveTransactionManager(hiveOps, fileSystemProvider, fileSystemExecutor); metadataOps = hiveOps; - - icebergHiveCatalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); } @Override @@ -337,10 +338,6 @@ public boolean isEnableHmsEventsIncrementalSync() { return enableHmsEventsIncrementalSync; } - public HiveCatalog getIcebergHiveCatalog() { - return icebergHiveCatalog; - } - /** * Enum for meta tables in hive catalog. * eg: tbl$partitions @@ -393,5 +390,14 @@ public TableValuedFunctionRef createFunctionRef(String ctlName, String dbName, S } } } + + public IcebergMetadataOps getIcebergMetadataOps() { + makeSureInitialized(); + if (icebergMetadataOps == null) { + HiveCatalog icebergHiveCatalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); + icebergMetadataOps = ExternalMetadataOperations.newIcebergMetadataOps(this, icebergHiveCatalog); + } + return icebergMetadataOps; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 6d65f8bcdbccb7..ad685386ec9e89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -17,6 +17,7 @@ package org.apache.doris.datasource.hive; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ListPartitionItem; @@ -31,6 +32,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; +import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.hudi.HudiUtils; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.datasource.mvcc.MvccSnapshot; @@ -41,6 +43,7 @@ import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVTimestampSnapshot; import org.apache.doris.nereids.exceptions.NotSupportedException; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.qe.GlobalVariable; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; @@ -301,12 +304,33 @@ public List getPartitionColumns(Optional snapshot) { } @Override - public boolean supportPartitionPruned() { - return getDlaType() == DLAType.HIVE; + public boolean supportInternalPartitionPruned() { + return getDlaType() == DLAType.HIVE || getDlaType() == DLAType.HUDI; + } + + public SelectedPartitions initHudiSelectedPartitions(Optional tableSnapshot) { + if (getDlaType() != DLAType.HUDI) { + return SelectedPartitions.NOT_PRUNED; + } + + if (getPartitionColumns().isEmpty()) { + return SelectedPartitions.NOT_PRUNED; + } + TablePartitionValues tablePartitionValues = HudiUtils.getPartitionValues(tableSnapshot, this); + + Map idToPartitionItem = tablePartitionValues.getIdToPartitionItem(); + Map idToNameMap = tablePartitionValues.getPartitionIdToNameMap(); + + Map nameToPartitionItems = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); + for (Entry entry : idToPartitionItem.entrySet()) { + nameToPartitionItems.put(idToNameMap.get(entry.getKey()), entry.getValue()); + } + + return new SelectedPartitions(nameToPartitionItems.size(), nameToPartitionItems, false); } @Override - public Map getNameToPartitionItems(Optional snapshot) { + protected Map getNameToPartitionItems(Optional snapshot) { return getNameToPartitionItems(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java index 97032467cec765..884cfbee45ba9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClientHelper.java @@ -40,7 +40,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.DdlException; import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopUGI; +import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.fs.remote.dfs.DFSFileSystem; import org.apache.doris.thrift.TExprOpcode; @@ -64,9 +64,11 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.time.LocalDateTime; import java.time.ZoneId; @@ -94,6 +96,9 @@ public class HiveMetaStoreClientHelper { private static final Pattern digitPattern = Pattern.compile("(\\d+)"); + public static final String HIVE_JSON_SERDE = "org.apache.hive.hcatalog.data.JsonSerDe"; + public static final String LEGACY_HIVE_JSON_SERDE = "org.apache.hadoop.hive.serde2.JsonSerDe"; + public enum HiveFileFormat { TEXT_FILE(0, "text"), PARQUET(1, "parquet"), @@ -819,17 +824,22 @@ public static T ugiDoAs(long catalogId, PrivilegedExceptionAction action) public static T ugiDoAs(Configuration conf, PrivilegedExceptionAction action) { // if hive config is not ready, then use hadoop kerberos to login - AuthenticationConfig krbConfig = AuthenticationConfig.getKerberosConfig(conf, - AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL, - AuthenticationConfig.HADOOP_KERBEROS_KEYTAB); - return HadoopUGI.ugiDoAs(krbConfig, action); + AuthenticationConfig authenticationConfig = AuthenticationConfig.getKerberosConfig(conf); + HadoopAuthenticator hadoopAuthenticator = HadoopAuthenticator.getHadoopAuthenticator(authenticationConfig); + try { + return hadoopAuthenticator.doAs(action); + } catch (IOException e) { + LOG.warn("HiveMetaStoreClientHelper ugiDoAs failed.", e); + throw new RuntimeException(e); + } } public static HoodieTableMetaClient getHudiClient(HMSExternalTable table) { String hudiBasePath = table.getRemoteTable().getSd().getLocation(); Configuration conf = getConfiguration(table); - return HadoopUGI.ugiDoAs(AuthenticationConfig.getKerberosConfig(conf), - () -> HoodieTableMetaClient.builder().setConf(conf).setBasePath(hudiBasePath).build()); + HadoopStorageConfiguration hadoopStorageConfiguration = new HadoopStorageConfiguration(conf); + return ugiDoAs(conf, () -> HoodieTableMetaClient.builder().setConf(hadoopStorageConfiguration) + .setBasePath(hudiBasePath).build()); } public static Configuration getConfiguration(HMSExternalTable table) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index e710bdb935d7bc..99d3cd1cd21622 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -82,7 +82,7 @@ public class HiveScanNode extends FileQueryScanNode { // will only be set in Nereids, for lagency planner, it should be null @Setter - private SelectedPartitions selectedPartitions = null; + protected SelectedPartitions selectedPartitions = null; private boolean partitionInit = false; private final AtomicReference batchException = new AtomicReference<>(null); @@ -364,14 +364,21 @@ public TableIf getTargetTable() { @Override public TFileFormatType getFileFormatType() throws UserException { TFileFormatType type = null; - String inputFormatName = hmsTable.getRemoteTable().getSd().getInputFormat(); + Table table = hmsTable.getRemoteTable(); + String inputFormatName = table.getSd().getInputFormat(); String hiveFormat = HiveMetaStoreClientHelper.HiveFileFormat.getFormat(inputFormatName); if (hiveFormat.equals(HiveMetaStoreClientHelper.HiveFileFormat.PARQUET.getDesc())) { type = TFileFormatType.FORMAT_PARQUET; } else if (hiveFormat.equals(HiveMetaStoreClientHelper.HiveFileFormat.ORC.getDesc())) { type = TFileFormatType.FORMAT_ORC; } else if (hiveFormat.equals(HiveMetaStoreClientHelper.HiveFileFormat.TEXT_FILE.getDesc())) { - type = TFileFormatType.FORMAT_CSV_PLAIN; + String serDeLib = table.getSd().getSerdeInfo().getSerializationLib(); + if (serDeLib.equals(HiveMetaStoreClientHelper.HIVE_JSON_SERDE) + || serDeLib.equals(HiveMetaStoreClientHelper.LEGACY_HIVE_JSON_SERDE)) { + type = TFileFormatType.FORMAT_JSON; + } else { + type = TFileFormatType.FORMAT_CSV_PLAIN; + } } return type; } @@ -383,11 +390,12 @@ protected Map getLocationProperties() throws UserException { @Override protected TFileAttributes getFileAttributes() throws UserException { - TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); + TFileAttributes fileAttributes = new TFileAttributes(); Table table = hmsTable.getRemoteTable(); // TODO: separate hive text table and OpenCsv table String serDeLib = table.getSd().getSerdeInfo().getSerializationLib(); if (serDeLib.equals("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) { + TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); // set properties of LazySimpleSerDe // 1. set column separator textParams.setColumnSeparator(HiveProperties.getFieldDelimiter(table)); @@ -401,7 +409,10 @@ protected TFileAttributes getFileAttributes() throws UserException { HiveProperties.getEscapeDelimiter(table).ifPresent(d -> textParams.setEscape(d.getBytes()[0])); // 6. set null format textParams.setNullFormat(HiveProperties.getNullFormat(table)); + fileAttributes.setTextParams(textParams); + fileAttributes.setHeaderType(""); } else if (serDeLib.equals("org.apache.hadoop.hive.serde2.OpenCSVSerde")) { + TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); // set set properties of OpenCSVSerde // 1. set column separator textParams.setColumnSeparator(HiveProperties.getSeparatorChar(table)); @@ -411,17 +422,29 @@ protected TFileAttributes getFileAttributes() throws UserException { textParams.setEnclose(HiveProperties.getQuoteChar(table).getBytes()[0]); // 4. set escape char textParams.setEscape(HiveProperties.getEscapeChar(table).getBytes()[0]); + fileAttributes.setTextParams(textParams); + fileAttributes.setHeaderType(""); + if (textParams.isSetEnclose()) { + fileAttributes.setTrimDoubleQuotes(true); + } + } else if (serDeLib.equals("org.apache.hive.hcatalog.data.JsonSerDe")) { + TFileTextScanRangeParams textParams = new TFileTextScanRangeParams(); + textParams.setColumnSeparator("\t"); + textParams.setLineDelimiter("\n"); + fileAttributes.setTextParams(textParams); + + fileAttributes.setJsonpaths(""); + fileAttributes.setJsonRoot(""); + fileAttributes.setNumAsString(true); + fileAttributes.setFuzzyParse(false); + fileAttributes.setReadJsonByLine(true); + fileAttributes.setStripOuterArray(false); + fileAttributes.setHeaderType(""); } else { throw new UserException( "unsupported hive table serde: " + serDeLib); } - TFileAttributes fileAttributes = new TFileAttributes(); - fileAttributes.setTextParams(textParams); - fileAttributes.setHeaderType(""); - if (textParams.isSet(TFileTextScanRangeParams._Fields.ENCLOSE)) { - fileAttributes.setTrimDoubleQuotes(true); - } return fileAttributes; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java index d7803b1a516f9e..0f38abafaa4d98 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/HudiUtils.java @@ -17,24 +17,35 @@ package org.apache.doris.datasource.hudi; +import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.ArrayType; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; import org.apache.doris.catalog.Type; +import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; +import org.apache.doris.datasource.hudi.source.HudiCachedPartitionProcessor; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; public class HudiUtils { @@ -86,7 +97,7 @@ public static String convertAvroToHiveType(Schema schema) { case LONG: if (logicalType instanceof LogicalTypes.TimestampMillis || logicalType instanceof LogicalTypes.TimestampMicros) { - return logicalType.getName(); + return "timestamp"; } if (logicalType instanceof LogicalTypes.TimeMicros) { return handleUnsupportedType(schema); @@ -231,4 +242,43 @@ private static Type handleUnionType(Schema avroSchema) { } return Type.UNSUPPORTED; } + + public static TablePartitionValues getPartitionValues(Optional tableSnapshot, + HMSExternalTable hmsTable) { + TablePartitionValues partitionValues = new TablePartitionValues(); + if (hmsTable.getPartitionColumns().isEmpty()) { + //isn't partition table. + return partitionValues; + } + + HoodieTableMetaClient hudiClient = HiveMetaStoreClientHelper.getHudiClient(hmsTable); + HudiCachedPartitionProcessor processor = (HudiCachedPartitionProcessor) Env.getCurrentEnv() + .getExtMetaCacheMgr().getHudiPartitionProcess(hmsTable.getCatalog()); + boolean useHiveSyncPartition = hmsTable.useHiveSyncPartition(); + + if (tableSnapshot.isPresent()) { + if (tableSnapshot.get().getType() == TableSnapshot.VersionType.VERSION) { + // Hudi does not support `FOR VERSION AS OF`, please use `FOR TIME AS OF`"; + return partitionValues; + } + String queryInstant = tableSnapshot.get().getTime().replaceAll("[-: ]", ""); + + partitionValues = + HiveMetaStoreClientHelper.ugiDoAs( + HiveMetaStoreClientHelper.getConfiguration(hmsTable), + () -> processor.getSnapshotPartitionValues( + hmsTable, hudiClient, queryInstant, useHiveSyncPartition)); + } else { + HoodieTimeline timeline = hudiClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); + Option snapshotInstant = timeline.lastInstant(); + if (!snapshotInstant.isPresent()) { + return partitionValues; + } + partitionValues = + HiveMetaStoreClientHelper.ugiDoAs( + HiveMetaStoreClientHelper.getConfiguration(hmsTable), + () -> processor.getPartitionValues(hmsTable, hudiClient, useHiveSyncPartition)); + } + return partitionValues; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java index 7981a0b4f261ff..843dded27969ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java @@ -37,6 +37,7 @@ import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.storage.StoragePath; import java.io.IOException; import java.util.ArrayList; @@ -105,7 +106,7 @@ public COWIncrementalRelation(Map optParams, Configuration confi List commitsToReturn = commitsTimelineToReturn.getInstants(); // todo: support configuration hoodie.datasource.read.incr.filters - Path basePath = metaClient.getBasePathV2(); + StoragePath basePath = metaClient.getBasePathV2(); Map regularFileIdToFullPath = new HashMap<>(); Map metaBootstrapFileIdToFullPath = new HashMap<>(); HoodieTimeline replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline(); @@ -113,8 +114,8 @@ public COWIncrementalRelation(Map optParams, Configuration confi for (HoodieInstant instant : replacedTimeline.getInstants()) { HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class).getPartitionToReplaceFileIds().forEach( - (key, value) -> value.forEach( - e -> replacedFile.put(e, FSUtils.getPartitionPath(basePath, key).toString()))); + (key, value) -> value.forEach( + e -> replacedFile.put(e, FSUtils.constructAbsolutePath(basePath, key).toString()))); } fileToWriteStat = new HashMap<>(); @@ -123,7 +124,7 @@ public COWIncrementalRelation(Map optParams, Configuration confi commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); metadata.getPartitionToWriteStats().forEach((partition, stats) -> { for (HoodieWriteStat stat : stats) { - fileToWriteStat.put(FSUtils.getPartitionPath(basePath, stat.getPath()).toString(), stat); + fileToWriteStat.put(FSUtils.constructAbsolutePath(basePath, stat.getPath()).toString(), stat); } }); if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS.equals(commit.getTimestamp())) { @@ -158,7 +159,7 @@ public COWIncrementalRelation(Map optParams, Configuration confi } - fs = basePath.getFileSystem(configuration); + fs = new Path(basePath.toUri().getPath()).getFileSystem(configuration); fullTableScan = shouldFullTableScan(); includeStartTime = !fullTableScan; if (fullTableScan || commitsToReturn.isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiLocalEngineContext.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiLocalEngineContext.java index 26ef6fdfef7086..fecc026cf8d046 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiLocalEngineContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiLocalEngineContext.java @@ -17,10 +17,6 @@ package org.apache.doris.datasource.hudi.source; -import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.data.HoodieAccumulator; import org.apache.hudi.common.data.HoodieAtomicLongAccumulator; import org.apache.hudi.common.data.HoodieData; @@ -39,7 +35,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.storage.StorageConfiguration; import java.util.Collections; import java.util.Iterator; @@ -50,18 +46,20 @@ import java.util.stream.Stream; /** - * This file is copied from org.apache.hudi.common.engine.HoodieLocalEngineContext. + * This file is copied from + * org.apache.hudi.common.engine.HudiLocalEngineContext. * Because we need set ugi in thread pool - * A java based engine context, use this implementation on the query engine integrations if needed. + * A java based engine context, use this implementation on the query engine + * integrations if needed. */ public final class HudiLocalEngineContext extends HoodieEngineContext { - public HudiLocalEngineContext(Configuration conf) { + public HudiLocalEngineContext(StorageConfiguration conf) { this(conf, new LocalTaskContextSupplier()); } - public HudiLocalEngineContext(Configuration conf, TaskContextSupplier taskContextSupplier) { - super(new SerializableConfiguration(conf), taskContextSupplier); + public HudiLocalEngineContext(StorageConfiguration conf, TaskContextSupplier taskContextSupplier) { + super(conf, taskContextSupplier); } @Override @@ -81,27 +79,18 @@ public HoodieData parallelize(List data, int parallelism) { @Override public List map(List data, SerializableFunction func, int parallelism) { - return data.stream().parallel().map(v1 -> { - try { - return HiveMetaStoreClientHelper.ugiDoAs(getHadoopConf().get(), () -> func.apply(v1)); - } catch (Exception e) { - throw new HoodieException("Error occurs when executing map", e); - } - }).collect(Collectors.toList()); + return data.stream().parallel().map(FunctionWrapper.throwingMapWrapper(func)).collect(Collectors.toList()); } @Override public List mapToPairAndReduceByKey( - List data, - SerializablePairFunction mapToPairFunc, - SerializableBiFunction reduceFunc, int parallelism) { + List data, SerializablePairFunction mapToPairFunc, SerializableBiFunction reduceFunc, + int parallelism) { return data.stream().parallel().map(FunctionWrapper.throwingMapToPairWrapper(mapToPairFunc)) - .collect(Collectors.groupingBy(p -> p.getKey())).values().stream() - .map(list -> - list.stream() - .map(e -> e.getValue()) + .collect(Collectors.groupingBy(p -> p.getKey())).values().stream() + .map(list -> list.stream().map(e -> e.getValue()) .reduce(FunctionWrapper.throwingReduceWrapper(reduceFunc)).get()) - .collect(Collectors.toList()); + .collect(Collectors.toList()); } @Override @@ -109,29 +98,28 @@ public Stream> mapPartitionsToPairAndReduceByKey( Stream data, SerializablePairFlatMapFunction, K, V> flatMapToPairFunc, SerializableBiFunction reduceFunc, int parallelism) { return FunctionWrapper.throwingFlatMapToPairWrapper(flatMapToPairFunc).apply(data.parallel().iterator()) - .collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream() - .map(entry -> new ImmutablePair<>(entry.getKey(), entry.getValue().stream().map( - Pair::getValue).reduce(FunctionWrapper.throwingReduceWrapper(reduceFunc)).orElse(null))) - .filter(Objects::nonNull); + .collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream() + .map(entry -> new ImmutablePair<>(entry.getKey(), entry.getValue().stream().map( + Pair::getValue).reduce(FunctionWrapper.throwingReduceWrapper(reduceFunc)).orElse(null))) + .filter(Objects::nonNull); } @Override public List reduceByKey( List> data, SerializableBiFunction reduceFunc, int parallelism) { return data.stream().parallel() - .collect(Collectors.groupingBy(p -> p.getKey())).values().stream() - .map(list -> - list.stream() - .map(e -> e.getValue()) - .reduce(FunctionWrapper.throwingReduceWrapper(reduceFunc)).orElse(null)) - .filter(Objects::nonNull) - .collect(Collectors.toList()); + .collect(Collectors.groupingBy(p -> p.getKey())).values().stream() + .map(list -> list.stream().map(e -> e.getValue()) + .reduce(FunctionWrapper.throwingReduceWrapper(reduceFunc)) + .orElse(null)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); } @Override public List flatMap(List data, SerializableFunction> func, int parallelism) { - return - data.stream().parallel().flatMap(FunctionWrapper.throwingFlatMapWrapper(func)).collect(Collectors.toList()); + return data.stream().parallel().flatMap(FunctionWrapper.throwingFlatMapWrapper(func)) + .collect(Collectors.toList()); } @Override @@ -142,8 +130,7 @@ public void foreach(List data, SerializableConsumer consumer, int para @Override public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { return data.stream().map(FunctionWrapper.throwingMapToPairWrapper(func)).collect( - Collectors.toMap(Pair::getLeft, Pair::getRight, (oldVal, newVal) -> newVal) - ); + Collectors.toMap(Pair::getLeft, Pair::getRight, (oldVal, newVal) -> newVal)); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java index 738b2638588e03..0ab9fef951a378 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiPartitionProcessor.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineUtils; -import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataUtil; @@ -49,14 +48,15 @@ public List getAllPartitionNames(HoodieTableMetaClient tableMetaClient) .build(); HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create( - new HudiLocalEngineContext(tableMetaClient.getHadoopConf()), metadataConfig, + new HudiLocalEngineContext(tableMetaClient.getStorageConf()), tableMetaClient.getStorage(), + metadataConfig, tableMetaClient.getBasePathV2().toString(), true); return newTableMetadata.getAllPartitionPaths(); } public List getPartitionNamesBeforeOrEquals(HoodieTimeline timeline, String timestamp) { - return new ArrayList<>(HoodieInputFormatUtils.getWritePartitionPaths( + return new ArrayList<>(HoodieTableMetadataUtil.getWritePartitionPaths( timeline.findInstantsBeforeOrEquals(timestamp).getInstants().stream().map(instant -> { try { return TimelineUtils.getCommitMetadata(instant, timeline); @@ -67,7 +67,7 @@ public List getPartitionNamesBeforeOrEquals(HoodieTimeline timeline, Str } public List getPartitionNamesInRange(HoodieTimeline timeline, String startTimestamp, String endTimestamp) { - return new ArrayList<>(HoodieInputFormatUtils.getWritePartitionPaths( + return new ArrayList<>(HoodieTableMetadataUtil.getWritePartitionPaths( timeline.findInstantsInRange(startTimestamp, endTimestamp).getInstants().stream().map(instant -> { try { return TimelineUtils.getCommitMetadata(instant, timeline); @@ -101,8 +101,10 @@ public static List parsePartitionValues(List partitionColumns, S } else { // If the partition column size is not equal to the partition fragments size // and the partition column size > 1, we do not know how to map the partition - // fragments to the partition columns and therefore return an empty tuple. We don't - // fail outright so that in some cases we can fallback to reading the table as non-partitioned + // fragments to the partition columns and therefore return an empty tuple. We + // don't + // fail outright so that in some cases we can fallback to reading the table as + // non-partitioned // one throw new RuntimeException("Failed to parse partition values of path: " + partitionPath); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java index a8f2a362bfde8d..b2cad8ab710178 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java @@ -21,22 +21,23 @@ import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ListPartitionItem; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.FileFormatUtils; import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.FileSplit; import org.apache.doris.datasource.TableFormatType; -import org.apache.doris.datasource.TablePartitionValues; import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper; import org.apache.doris.datasource.hive.HivePartition; import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.hudi.HudiUtils; -import org.apache.doris.planner.ListPartitionPrunerV2; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; import org.apache.doris.spi.Split; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TExplainLevel; @@ -48,8 +49,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BaseFile; @@ -62,13 +61,14 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Locale; @@ -87,7 +87,7 @@ public class HudiScanNode extends HiveScanNode { private static final Logger LOG = LogManager.getLogger(HudiScanNode.class); - private boolean isCowOrRoTable; + private boolean isCowTable; private final AtomicLong noLogsSplitNum = new AtomicLong(0); @@ -113,19 +113,23 @@ public class HudiScanNode extends HiveScanNode { private boolean incrementalRead = false; private TableScanParams scanParams; private IncrementalRelation incrementalRelation; + private SessionVariable sessionVariable; /** * External file scan node for Query Hudi table - * needCheckColumnPriv: Some of ExternalFileScanNode do not need to check column priv + * needCheckColumnPriv: Some of ExternalFileScanNode do not need to check column + * priv * eg: s3 tvf - * These scan nodes do not have corresponding catalog/database/table info, so no need to do priv check + * These scan nodes do not have corresponding catalog/database/table info, so no + * need to do priv check */ public HudiScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv, - Optional scanParams, Optional incrementalRelation) { + Optional scanParams, Optional incrementalRelation, + SessionVariable sessionVariable) { super(id, desc, "HUDI_SCAN_NODE", StatisticalType.HUDI_SCAN_NODE, needCheckColumnPriv); - isCowOrRoTable = hmsTable.isHoodieCowTable(); + isCowTable = hmsTable.isHoodieCowTable(); if (LOG.isDebugEnabled()) { - if (isCowOrRoTable) { + if (isCowTable) { LOG.debug("Hudi table {} can read as cow/read optimize table", hmsTable.getFullQualifiers()); } else { LOG.debug("Hudi table {} is a mor table, and will use JNI to read data in BE", @@ -136,11 +140,12 @@ public HudiScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumn this.scanParams = scanParams.orElse(null); this.incrementalRelation = incrementalRelation.orElse(null); this.incrementalRead = (this.scanParams != null && this.scanParams.incrementalRead()); + this.sessionVariable = sessionVariable; } @Override public TFileFormatType getFileFormatType() throws UserException { - if (isCowOrRoTable) { + if (canUseNativeReader()) { return super.getFileFormatType(); } else { // Use jni to read hudi table in BE @@ -185,13 +190,13 @@ protected void doInitialize() throws UserException { throw new UserException("Not support function '" + scanParams.getParamType() + "' in hudi table"); } if (incrementalRead) { - if (isCowOrRoTable) { + if (isCowTable) { try { Map serd = hmsTable.getRemoteTable().getSd().getSerdeInfo().getParameters(); if ("true".equals(serd.get("hoodie.query.as.ro.table")) && hmsTable.getRemoteTable().getTableName().endsWith("_ro")) { // Incremental read RO table as RT table, I don't know why? - isCowOrRoTable = false; + isCowTable = false; LOG.warn("Execute incremental read on RO table: {}", hmsTable.getFullQualifiers()); } } catch (Exception e) { @@ -236,7 +241,22 @@ protected Map getLocationProperties() throws UserException { @Override protected void setScanParams(TFileRangeDesc rangeDesc, Split split) { if (split instanceof HudiSplit) { - setHudiParams(rangeDesc, (HudiSplit) split); + HudiSplit hudiSplit = (HudiSplit) split; + if (rangeDesc.getFormatType() == TFileFormatType.FORMAT_JNI + && !sessionVariable.isForceJniScanner() + && hudiSplit.getHudiDeltaLogs().isEmpty()) { + // no logs, is read optimize table, fallback to use native reader + String fileFormat = FileFormatUtils.getFileFormatBySuffix(hudiSplit.getDataFilePath()) + .orElse("Unknown"); + if (fileFormat.equals("parquet")) { + rangeDesc.setFormatType(TFileFormatType.FORMAT_PARQUET); + } else if (fileFormat.equals("orc")) { + rangeDesc.setFormatType(TFileFormatType.FORMAT_ORC); + } else { + throw new RuntimeException("Unsupported file format: " + fileFormat); + } + } + setHudiParams(rangeDesc, hudiSplit); } } @@ -255,56 +275,41 @@ private void setHudiParams(TFileRangeDesc rangeDesc, HudiSplit hudiSplit) { fileDesc.setColumnTypes(hudiSplit.getHudiColumnTypes()); // TODO(gaoxin): support complex types // fileDesc.setNestedFields(hudiSplit.getNestedFields()); + fileDesc.setHudiJniScanner(hudiSplit.getHudiJniScanner()); tableFormatFileDesc.setHudiParams(fileDesc); rangeDesc.setTableFormatParams(tableFormatFileDesc); } - private List getPrunedPartitions( - HoodieTableMetaClient metaClient, Option snapshotTimestamp) throws AnalysisException { + private boolean canUseNativeReader() { + return !sessionVariable.isForceJniScanner() && isCowTable; + } + + private List getPrunedPartitions(HoodieTableMetaClient metaClient) { List partitionColumnTypes = hmsTable.getPartitionColumnTypes(); if (!partitionColumnTypes.isEmpty()) { - HudiCachedPartitionProcessor processor = (HudiCachedPartitionProcessor) Env.getCurrentEnv() - .getExtMetaCacheMgr().getHudiPartitionProcess(hmsTable.getCatalog()); - TablePartitionValues partitionValues; - if (snapshotTimestamp.isPresent()) { - partitionValues = processor.getSnapshotPartitionValues( - hmsTable, metaClient, snapshotTimestamp.get(), useHiveSyncPartition); - } else { - partitionValues = processor.getPartitionValues(hmsTable, metaClient, useHiveSyncPartition); - } - if (partitionValues != null) { - // 2. prune partitions by expr - partitionValues.readLock().lock(); - try { - Map idToPartitionItem = partitionValues.getIdToPartitionItem(); - this.totalPartitionNum = idToPartitionItem.size(); - ListPartitionPrunerV2 pruner = new ListPartitionPrunerV2(idToPartitionItem, - hmsTable.getPartitionColumns(), columnNameToRange, - partitionValues.getUidToPartitionRange(), - partitionValues.getRangeToId(), - partitionValues.getSingleColumnRangeMap(), - true); - Collection filteredPartitionIds = pruner.prune(); - this.selectedPartitionNum = filteredPartitionIds.size(); - // 3. get partitions from cache - String dbName = hmsTable.getDbName(); - String tblName = hmsTable.getName(); - String inputFormat = hmsTable.getRemoteTable().getSd().getInputFormat(); - String basePath = metaClient.getBasePathV2().toString(); - Map partitionIdToNameMap = partitionValues.getPartitionIdToNameMap(); - Map> partitionValuesMap = partitionValues.getPartitionValuesMap(); - return filteredPartitionIds.stream().map(id -> { - String path = basePath + "/" + partitionIdToNameMap.get(id); - return new HivePartition( - dbName, tblName, false, inputFormat, path, partitionValuesMap.get(id), - Maps.newHashMap()); - }).collect(Collectors.toList()); - } finally { - partitionValues.readLock().unlock(); - } - } + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + Map prunedPartitions = selectedPartitions.selectedPartitions; + this.selectedPartitionNum = prunedPartitions.size(); + + String dbName = hmsTable.getDbName(); + String tblName = hmsTable.getName(); + String inputFormat = hmsTable.getRemoteTable().getSd().getInputFormat(); + String basePath = metaClient.getBasePathV2().toString(); + + List hivePartitions = Lists.newArrayList(); + prunedPartitions.forEach( + (key, value) -> { + String path = basePath + "/" + key; + hivePartitions.add(new HivePartition( + dbName, tblName, false, inputFormat, path, + ((ListPartitionItem) value).getItems().get(0).getPartitionValuesAsStringList(), + Maps.newHashMap())); + } + ); + return hivePartitions; } - // unpartitioned table, create a dummy partition to save location and inputformat, + // unpartitioned table, create a dummy partition to save location and + // inputformat, // so that we can unify the interface. HivePartition dummyPartition = new HivePartition(hmsTable.getDbName(), hmsTable.getName(), true, hmsTable.getRemoteTable().getSd().getInputFormat(), @@ -315,7 +320,7 @@ private List getPrunedPartitions( } private List getIncrementalSplits() { - if (isCowOrRoTable) { + if (canUseNativeReader()) { List splits = incrementalRelation.collectSplits(); noLogsSplitNum.addAndGet(splits.size()); return splits; @@ -336,15 +341,15 @@ private void getPartitionSplits(HivePartition partition, List splits) thr globPath = hudiClient.getBasePathV2().toString() + "/*"; } else { partitionName = FSUtils.getRelativePartitionPath(hudiClient.getBasePathV2(), - new Path(partition.getPath())); + new StoragePath(partition.getPath())); globPath = String.format("%s/%s/*", hudiClient.getBasePathV2().toString(), partitionName); } - List statuses = FSUtils.getGlobStatusExcludingMetaFolder( - hudiClient.getRawFs(), new Path(globPath)); + List statuses = FSUtils.getGlobStatusExcludingMetaFolder( + hudiClient.getRawHoodieStorage(), new StoragePath(globPath)); HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(hudiClient, - timeline, statuses.toArray(new FileStatus[0])); + timeline, statuses); - if (isCowOrRoTable) { + if (canUseNativeReader()) { fileSystemView.getLatestBaseFilesBeforeOrOn(partitionName, queryInstant).forEach(baseFile -> { noLogsSplitNum.incrementAndGet(); String filePath = baseFile.getPath(); @@ -392,7 +397,7 @@ public List getSplits() throws UserException { if (!partitionInit) { prunedPartitions = HiveMetaStoreClientHelper.ugiDoAs( HiveMetaStoreClientHelper.getConfiguration(hmsTable), - () -> getPrunedPartitions(hudiClient, snapshotTimestamp)); + () -> getPrunedPartitions(hudiClient)); partitionInit = true; } List splits = Collections.synchronizedList(new ArrayList<>()); @@ -454,7 +459,7 @@ public boolean isBatchMode() { // Non partition table will get one dummy partition prunedPartitions = HiveMetaStoreClientHelper.ugiDoAs( HiveMetaStoreClientHelper.getConfiguration(hmsTable), - () -> getPrunedPartitions(hudiClient, snapshotTimestamp)); + () -> getPrunedPartitions(hudiClient)); partitionInit = true; } int numPartitions = ConnectContext.get().getSessionVariable().getNumPartitionsInBatchMode(); @@ -473,9 +478,9 @@ private HudiSplit generateHudiSplit(FileSlice fileSlice, List partitionV fileSlice.getPartitionPath(); List logs = fileSlice.getLogFiles().map(HoodieLogFile::getPath) - .map(Path::toString) + .map(StoragePath::toString) .collect(Collectors.toList()); - if (logs.isEmpty()) { + if (logs.isEmpty() && !sessionVariable.isForceJniScanner()) { noLogsSplitNum.incrementAndGet(); } @@ -492,6 +497,7 @@ private HudiSplit generateHudiSplit(FileSlice fileSlice, List partitionV split.setHudiColumnNames(columnNames); split.setHudiColumnTypes(columnTypes); split.setInstantTime(queryInstant); + split.setHudiJniScanner(sessionVariable.getHudiJniScanner()); return split; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiSplit.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiSplit.java index c72f7621feaa55..2270d2017937da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiSplit.java @@ -40,6 +40,5 @@ public HudiSplit(LocationPath file, long start, long length, long fileLength, St private List hudiColumnNames; private List hudiColumnTypes; private List nestedFields; + private String hudiJniScanner; } - - diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java index c06fcc2a578d43..7df013599229fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java @@ -20,9 +20,7 @@ import org.apache.doris.spi.Split; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.GlobPattern; -import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -34,6 +32,8 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.storage.StoragePathInfo; import java.io.IOException; import java.util.ArrayList; @@ -54,7 +54,7 @@ public class MORIncrementalRelation implements IncrementalRelation { private final boolean endInstantArchived; private final List includedCommits; private final List commitsMetadata; - private final FileStatus[] affectedFilesInCommits; + private final List affectedFilesInCommits; private final boolean fullTableScan; private final String globPattern; private final boolean includeStartTime; @@ -96,7 +96,7 @@ public MORIncrementalRelation(Map optParams, Configuration confi includedCommits = getIncludedCommits(); commitsMetadata = getCommitsMetadata(); affectedFilesInCommits = HoodieInputFormatUtils.listAffectedFilesForCommits(configuration, - new Path(metaClient.getBasePath()), commitsMetadata); + metaClient.getBasePathV2(), commitsMetadata); fullTableScan = shouldFullTableScan(); if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME && fullTableScan) { throw new HoodieException("Cannot use stateTransitionTime while enables full table scan"); @@ -152,8 +152,8 @@ private boolean shouldFullTableScan() throws IOException { if (should) { return true; } - for (FileStatus fileStatus : affectedFilesInCommits) { - if (!metaClient.getFs().exists(fileStatus.getPath())) { + for (StoragePathInfo fileStatus : affectedFilesInCommits) { + if (!metaClient.getRawHoodieStorage().exists(fileStatus.getPath())) { return true; } } @@ -199,7 +199,7 @@ public List collectFileSlices() throws HoodieException { String latestCommit = includedCommits.get(includedCommits.size() - 1).getTimestamp(); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, scanTimeline, affectedFilesInCommits); - Stream fileSlices = HoodieInputFormatUtils.getWritePartitionPaths(commitsMetadata) + Stream fileSlices = HoodieTableMetadataUtil.getWritePartitionPaths(commitsMetadata) .stream().flatMap(relativePartitionPath -> fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, latestCommit)); if ("".equals(globPattern)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java index d8dfd1c128f162..0fa69825a01ef4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java @@ -40,11 +40,10 @@ public abstract class IcebergExternalCatalog extends ExternalCatalog { public static final String ICEBERG_HADOOP = "hadoop"; public static final String ICEBERG_GLUE = "glue"; public static final String ICEBERG_DLF = "dlf"; + public static final String EXTERNAL_CATALOG_NAME = "external_catalog.name"; protected String icebergCatalogType; protected Catalog catalog; - protected PreExecutionAuthenticator preExecutionAuthenticator; - public IcebergExternalCatalog(long catalogId, String name, String comment) { super(catalogId, name, InitCatalogLog.Type.ICEBERG, comment); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java index c1ac2a79754b79..ad347ca78f2a4f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java @@ -36,8 +36,6 @@ import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; import java.util.HashMap; @@ -104,18 +102,16 @@ private List loadSnapshots(IcebergMetadataCacheKey key) { @NotNull private Table loadTable(IcebergMetadataCacheKey key) { - Catalog icebergCatalog; + IcebergMetadataOps ops; if (key.catalog instanceof HMSExternalCatalog) { - icebergCatalog = ((HMSExternalCatalog) key.catalog).getIcebergHiveCatalog(); + ops = ((HMSExternalCatalog) key.catalog).getIcebergMetadataOps(); } else if (key.catalog instanceof IcebergExternalCatalog) { - icebergCatalog = ((IcebergExternalCatalog) key.catalog).getCatalog(); + ops = (IcebergMetadataOps) (((IcebergExternalCatalog) key.catalog).getMetadataOps()); } else { throw new RuntimeException("Only support 'hms' and 'iceberg' type for iceberg table"); } - Table icebergTable = HiveMetaStoreClientHelper.ugiDoAs(((ExternalCatalog) key.catalog).getConfiguration(), - () -> icebergCatalog.loadTable(TableIdentifier.of(key.dbName, key.tableName))); - initIcebergTableFileIO(icebergTable, key.catalog.getProperties()); - return icebergTable; + return HiveMetaStoreClientHelper.ugiDoAs(((ExternalCatalog) key.catalog).getConfiguration(), + () -> ops.loadTable(key.dbName, key.tableName)); } public void invalidateCatalogCache(long catalogId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java index 59729ddb47a6a4..970814b7acdc85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java @@ -36,6 +36,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -46,29 +47,40 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; public class IcebergMetadataOps implements ExternalMetadataOps { private static final Logger LOG = LogManager.getLogger(IcebergMetadataOps.class); protected Catalog catalog; - protected IcebergExternalCatalog dorisCatalog; + protected ExternalCatalog dorisCatalog; protected SupportsNamespaces nsCatalog; private PreExecutionAuthenticator preExecutionAuthenticator; + // Generally, there should be only two levels under the catalog, namely ., + // but the REST type catalog is obtained from an external server, + // and the level provided by the external server may be three levels, ..
. + // Therefore, if the external server provides a catalog, + // the catalog needs to be recorded here to ensure semantic consistency. + private Optional externalCatalogName = Optional.empty(); - public IcebergMetadataOps(IcebergExternalCatalog dorisCatalog, Catalog catalog) { + public IcebergMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { this.dorisCatalog = dorisCatalog; this.catalog = catalog; nsCatalog = (SupportsNamespaces) catalog; - this.preExecutionAuthenticator = dorisCatalog.preExecutionAuthenticator; + this.preExecutionAuthenticator = dorisCatalog.getPreExecutionAuthenticator(); + if (dorisCatalog.getProperties().containsKey(IcebergExternalCatalog.EXTERNAL_CATALOG_NAME)) { + externalCatalogName = + Optional.of(dorisCatalog.getProperties().get(IcebergExternalCatalog.EXTERNAL_CATALOG_NAME)); + } } public Catalog getCatalog() { return catalog; } - public IcebergExternalCatalog getExternalCatalog() { + public ExternalCatalog getExternalCatalog() { return dorisCatalog; } @@ -78,17 +90,18 @@ public void close() { @Override public boolean tableExist(String dbName, String tblName) { - return catalog.tableExists(TableIdentifier.of(dbName, tblName)); + return catalog.tableExists(getTableIdentifier(dbName, tblName)); } public boolean databaseExist(String dbName) { - return nsCatalog.namespaceExists(Namespace.of(dbName)); + return nsCatalog.namespaceExists(getNamespace(dbName)); } public List listDatabaseNames() { try { - return preExecutionAuthenticator.execute(() -> nsCatalog.listNamespaces().stream() - .map(Namespace::toString) + return preExecutionAuthenticator.execute(() -> nsCatalog.listNamespaces(getNamespace()) + .stream() + .map(n -> n.level(n.length() - 1)) .collect(Collectors.toList())); } catch (Exception e) { throw new RuntimeException("Failed to list database names, error message is: " + e.getMessage()); @@ -98,7 +111,7 @@ public List listDatabaseNames() { @Override public List listTableNames(String dbName) { - List tableIdentifiers = catalog.listTables(Namespace.of(dbName)); + List tableIdentifiers = catalog.listTables(getNamespace(dbName)); return tableIdentifiers.stream().map(TableIdentifier::name).collect(Collectors.toList()); } @@ -128,12 +141,14 @@ private void performCreateDb(CreateDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_CREATE_EXISTS, dbName); } } - String icebergCatalogType = dorisCatalog.getIcebergCatalogType(); - if (!properties.isEmpty() && !IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { - throw new DdlException( + if (!properties.isEmpty() && dorisCatalog instanceof IcebergExternalCatalog) { + String icebergCatalogType = ((IcebergExternalCatalog) dorisCatalog).getIcebergCatalogType(); + if (!IcebergExternalCatalog.ICEBERG_HMS.equals(icebergCatalogType)) { + throw new DdlException( "Not supported: create database with properties for iceberg catalog type: " + icebergCatalogType); + } } - nsCatalog.createNamespace(Namespace.of(dbName), properties); + nsCatalog.createNamespace(getNamespace(dbName), properties); dorisCatalog.onRefreshCache(true); } @@ -159,8 +174,7 @@ private void preformDropDb(DropDbStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); } } - SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; - nsCatalog.dropNamespace(Namespace.of(dbName)); + nsCatalog.dropNamespace(getNamespace(dbName)); dorisCatalog.onRefreshCache(true); } @@ -200,7 +214,7 @@ public boolean performCreateTable(CreateTableStmt stmt) throws UserException { Map properties = stmt.getProperties(); properties.put(ExternalCatalog.DORIS_VERSION, ExternalCatalog.DORIS_VERSION_VALUE); PartitionSpec partitionSpec = IcebergUtils.solveIcebergPartitionSpec(stmt.getPartitionDesc(), schema); - catalog.createTable(TableIdentifier.of(dbName, tableName), schema, partitionSpec, properties); + catalog.createTable(getTableIdentifier(dbName, tableName), schema, partitionSpec, properties); db.setUnInitialized(true); return false; } @@ -238,7 +252,7 @@ private void performDropTable(DropTableStmt stmt) throws DdlException { ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName); } } - catalog.dropTable(TableIdentifier.of(dbName, tableName), true); + catalog.dropTable(getTableIdentifier(dbName, tableName), true); db.setUnInitialized(true); } @@ -250,4 +264,25 @@ public void truncateTable(String dbName, String tblName, List partitions public PreExecutionAuthenticator getPreExecutionAuthenticator() { return preExecutionAuthenticator; } + + @Override + public Table loadTable(String dbName, String tblName) { + return catalog.loadTable(getTableIdentifier(dbName, tblName)); + } + + private TableIdentifier getTableIdentifier(String dbName, String tblName) { + return externalCatalogName + .map(s -> TableIdentifier.of(s, dbName, tblName)) + .orElseGet(() -> TableIdentifier.of(dbName, tblName)); + } + + private Namespace getNamespace(String dbName) { + return externalCatalogName + .map(s -> Namespace.of(s, dbName)) + .orElseGet(() -> Namespace.of(dbName)); + } + + private Namespace getNamespace() { + return externalCatalogName.map(Namespace::of).orElseGet(() -> Namespace.empty()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java index 908a4fa9e3f271..b92d2c91f9630e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergRestExternalCatalog.java @@ -26,7 +26,6 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; -import org.apache.iceberg.aws.s3.S3FileIO; import org.apache.iceberg.aws.s3.S3FileIOProperties; import java.util.HashMap; @@ -71,7 +70,6 @@ private Map convertToRestCatalogProperties() { Map props = catalogProperty.getProperties(); Map restProperties = new HashMap<>(props); - restProperties.put(CatalogProperties.FILE_IO_IMPL, S3FileIO.class.getName()); restProperties.put(CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); String restUri = props.getOrDefault(CatalogProperties.URI, ""); restProperties.put(CatalogProperties.URI, restUri); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index 685915025d665e..d0cca11b0af2bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -22,6 +22,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.info.SimpleTableInfo; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.iceberg.helper.IcebergWriterHelper; import org.apache.doris.nereids.trees.plans.commands.insert.BaseExternalTableInsertCommandContext; import org.apache.doris.nereids.trees.plans.commands.insert.InsertCommandContext; @@ -140,7 +141,7 @@ public long getUpdateCnt() { private synchronized Table getNativeTable(SimpleTableInfo tableInfo) { Objects.requireNonNull(tableInfo); - IcebergExternalCatalog externalCatalog = ops.getExternalCatalog(); + ExternalCatalog externalCatalog = ops.getExternalCatalog(); return IcebergUtils.getRemoteTable(externalCatalog, tableInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java index 9e188a711b0297..772c366850b9a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalTable.java @@ -63,6 +63,9 @@ public class JdbcExternalTable extends ExternalTable { public static final String SQLSERVER_ROW_COUNT_SQL = "SELECT sum(rows) as rows FROM sys.partitions " + "WHERE object_id = (SELECT object_id('${dbName}.${tblName}')) AND index_id IN (0, 1)"; + public static final String ORACLE_ROW_COUNT_SQL = "SELECT NUM_ROWS as \\\"rows\\\" FROM ALL_TABLES WHERE " + + "OWNER = '${dbName}' and TABLE_NAME = '${tblName}'"; + public static final String FETCH_ROW_COUNT_TEMPLATE = "SELECT * FROM QUERY" + "(\"catalog\"=\"${ctlName}\", \"query\"=\"${sql}\");"; @@ -144,6 +147,8 @@ public long fetchRowCount() { params.put("sql", SQLSERVER_ROW_COUNT_SQL); return getRowCount(params); case JdbcResource.ORACLE: + params.put("sql", ORACLE_ROW_COUNT_SQL); + return getRowCount(params); default: break; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java index e6cd77103dbc3b..27e4976aedd85e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java @@ -33,6 +33,7 @@ import com.aliyun.odps.account.Account; import com.aliyun.odps.account.AliyunAccount; import com.aliyun.odps.security.SecurityManager; +import com.aliyun.odps.table.configuration.RestOptions; import com.aliyun.odps.table.configuration.SplitOptions; import com.aliyun.odps.table.enviroment.Credentials; import com.aliyun.odps.table.enviroment.EnvironmentSettings; @@ -71,6 +72,10 @@ public class MaxComputeExternalCatalog extends ExternalCatalog { private long splitRowCount; private long splitByteSize; + private int connectTimeout; + private int readTimeout; + private int retryTimes; + private static final Map REGION_ZONE_MAP; private static final List REQUIRED_PROPERTIES = ImmutableList.of( MCProperties.PROJECT, @@ -159,25 +164,38 @@ protected void initLocalObjectsImpl() { defaultProject = props.get(MCProperties.PROJECT); quota = props.getOrDefault(MCProperties.QUOTA, MCProperties.DEFAULT_QUOTA); + boolean splitCrossPartition = + Boolean.parseBoolean(props.getOrDefault(MCProperties.SPLIT_CROSS_PARTITION, + MCProperties.DEFAULT_SPLIT_CROSS_PARTITION)); splitStrategy = props.getOrDefault(MCProperties.SPLIT_STRATEGY, MCProperties.DEFAULT_SPLIT_STRATEGY); if (splitStrategy.equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { splitByteSize = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_BYTE_SIZE, MCProperties.DEFAULT_SPLIT_BYTE_SIZE)); - splitOptions = SplitOptions.newBuilder() .SplitByByteSize(splitByteSize) - .withCrossPartition(false) + .withCrossPartition(splitCrossPartition) .build(); } else { splitRowCount = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_ROW_COUNT, MCProperties.DEFAULT_SPLIT_ROW_COUNT)); splitOptions = SplitOptions.newBuilder() .SplitByRowOffset() - .withCrossPartition(false) + .withCrossPartition(splitCrossPartition) .build(); } + connectTimeout = Integer.parseInt( + props.getOrDefault(MCProperties.CONNECT_TIMEOUT, MCProperties.DEFAULT_CONNECT_TIMEOUT)); + readTimeout = Integer.parseInt( + props.getOrDefault(MCProperties.READ_TIMEOUT, MCProperties.DEFAULT_READ_TIMEOUT)); + retryTimes = Integer.parseInt( + props.getOrDefault(MCProperties.RETRY_COUNT, MCProperties.DEFAULT_RETRY_COUNT)); + + RestOptions restOptions = RestOptions.newBuilder() + .withConnectTimeout(connectTimeout) + .withReadTimeout(readTimeout) + .withRetryTimes(retryTimes).build(); CloudCredential credential = MCProperties.getCredential(props); accessKey = credential.getAccessKey(); @@ -196,6 +214,7 @@ protected void initLocalObjectsImpl() { .withCredentials(credentials) .withServiceEndpoint(odps.getEndpoint()) .withQuotaName(quota) + .withRestOptions(restOptions) .build(); } @@ -304,6 +323,21 @@ public String getDefaultProject() { return defaultProject; } + public int getRetryTimes() { + makeSureInitialized(); + return retryTimes; + } + + public int getConnectTimeout() { + makeSureInitialized(); + return connectTimeout; + } + + public int getReadTimeout() { + makeSureInitialized(); + return readTimeout; + } + public ZoneId getProjectDateTimeZone() { makeSureInitialized(); @@ -385,6 +419,31 @@ public void checkProperties() throws DdlException { + MCProperties.SPLIT_ROW_COUNT + "must be an integer"); } + + try { + connectTimeout = Integer.parseInt( + props.getOrDefault(MCProperties.CONNECT_TIMEOUT, MCProperties.DEFAULT_CONNECT_TIMEOUT)); + readTimeout = Integer.parseInt( + props.getOrDefault(MCProperties.READ_TIMEOUT, MCProperties.DEFAULT_READ_TIMEOUT)); + retryTimes = Integer.parseInt( + props.getOrDefault(MCProperties.RETRY_COUNT, MCProperties.DEFAULT_RETRY_COUNT)); + if (connectTimeout <= 0) { + throw new DdlException(MCProperties.CONNECT_TIMEOUT + " must be greater than 0"); + } + + if (readTimeout <= 0) { + throw new DdlException(MCProperties.READ_TIMEOUT + " must be greater than 0"); + } + + if (retryTimes <= 0) { + throw new DdlException(MCProperties.RETRY_COUNT + " must be greater than 0"); + } + + } catch (NumberFormatException e) { + throw new DdlException("property " + MCProperties.CONNECT_TIMEOUT + "/" + + MCProperties.READ_TIMEOUT + "/" + MCProperties.RETRY_COUNT + "must be an integer"); + } + CloudCredential credential = MCProperties.getCredential(props); if (!credential.isWhole()) { throw new DdlException("Max-Compute credential properties '" diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java index dc3232f79f5f71..0f748f59e927bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MapType; +import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; @@ -28,6 +29,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.thrift.TMCTable; import org.apache.doris.thrift.TTableDescriptor; import org.apache.doris.thrift.TTableType; @@ -50,6 +52,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.stream.Collectors; @@ -71,6 +74,15 @@ protected synchronized void makeSureInitialized() { } } + @Override + public boolean supportInternalPartitionPruned() { + return true; + } + + @Override + public List getPartitionColumns(Optional snapshot) { + return getPartitionColumns(); + } public List getPartitionColumns() { makeSureInitialized(); @@ -79,7 +91,24 @@ public List getPartitionColumns() { .orElse(Collections.emptyList()); } - public TablePartitionValues getPartitionValues() { + @Override + protected Map getNameToPartitionItems(Optional snapshot) { + if (getPartitionColumns().isEmpty()) { + return Collections.emptyMap(); + } + + TablePartitionValues tablePartitionValues = getPartitionValues(); + Map idToPartitionItem = tablePartitionValues.getIdToPartitionItem(); + Map idToNameMap = tablePartitionValues.getPartitionIdToNameMap(); + + Map nameToPartitionItem = Maps.newHashMapWithExpectedSize(idToPartitionItem.size()); + for (Entry entry : idToPartitionItem.entrySet()) { + nameToPartitionItem.put(idToNameMap.get(entry.getKey()), entry.getValue()); + } + return nameToPartitionItem; + } + + private TablePartitionValues getPartitionValues() { makeSureInitialized(); Optional schemaCacheValue = getSchemaCacheValue(); if (!schemaCacheValue.isPresent()) { @@ -110,6 +139,8 @@ private TablePartitionValues loadPartitionValues(MaxComputeSchemaCacheValue sche /** * parse all values from partitionPath to a single list. + * In MaxCompute : Support special characters : _$#.!@ + * Ref : MaxCompute Error Code: ODPS-0130071 Invalid partition value. * * @param partitionColumns partitionColumns can contain the part1,part2,part3... * @param partitionPath partitionPath format is like the 'part1=123/part2=abc/part3=1bc' diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java index e0b84b0860e551..9fa22a0fffaab8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java @@ -29,6 +29,7 @@ import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; @@ -40,8 +41,10 @@ import org.apache.doris.datasource.maxcompute.MaxComputeExternalTable; import org.apache.doris.datasource.maxcompute.source.MaxComputeSplit.SplitType; import org.apache.doris.datasource.property.constants.MCProperties; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.nereids.util.DateUtils; import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.spi.Split; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TFileFormatType; @@ -50,6 +53,7 @@ import org.apache.doris.thrift.TTableFormatFileDesc; import com.aliyun.odps.OdpsType; +import com.aliyun.odps.PartitionSpec; import com.aliyun.odps.table.TableIdentifier; import com.aliyun.odps.table.configuration.ArrowOptions; import com.aliyun.odps.table.configuration.ArrowOptions.TimestampUnit; @@ -60,6 +64,7 @@ import com.aliyun.odps.table.read.split.impl.IndexedInputSplit; import com.google.common.collect.Maps; import jline.internal.Log; +import lombok.Setter; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -76,24 +81,49 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; public class MaxComputeScanNode extends FileQueryScanNode { private final MaxComputeExternalTable table; - private TableBatchReadSession tableBatchReadSession; private Predicate filterPredicate; + List requiredPartitionColumns = new ArrayList<>(); + List orderedRequiredDataColumns = new ArrayList<>(); + + private int connectTimeout; + private int readTimeout; + private int retryTimes; + + @Setter + private SelectedPartitions selectedPartitions = null; + private static final LocationPath ROW_OFFSET_PATH = new LocationPath("/row_offset", Maps.newHashMap()); private static final LocationPath BYTE_SIZE_PATH = new LocationPath("/byte_size", Maps.newHashMap()); + + // For new planner + public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, + SelectedPartitions selectedPartitions, boolean needCheckColumnPriv) { + this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, + selectedPartitions, needCheckColumnPriv); + } + + // For old planner public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv) { - this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, needCheckColumnPriv); + this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, + SelectedPartitions.NOT_PRUNED, needCheckColumnPriv); } - public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, - StatisticalType statisticalType, boolean needCheckColumnPriv) { + private MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, + StatisticalType statisticalType, SelectedPartitions selectedPartitions, + boolean needCheckColumnPriv) { super(id, desc, planNodeName, statisticalType, needCheckColumnPriv); table = (MaxComputeExternalTable) desc.getTable(); + this.selectedPartitions = selectedPartitions; } @Override @@ -110,6 +140,11 @@ private void setScanParams(TFileRangeDesc rangeDesc, MaxComputeSplit maxComputeS fileDesc.setPartitionSpec("deprecated"); fileDesc.setTableBatchReadSession(maxComputeSplit.scanSerialize); fileDesc.setSessionId(maxComputeSplit.getSessionId()); + + fileDesc.setReadTimeout(readTimeout); + fileDesc.setConnectTimeout(connectTimeout); + fileDesc.setRetryTimes(retryTimes); + tableFormatFileDesc.setMaxComputeParams(fileDesc); rangeDesc.setTableFormatParams(tableFormatFileDesc); rangeDesc.setPath("[ " + maxComputeSplit.getStart() + " , " + maxComputeSplit.getLength() + " ]"); @@ -117,16 +152,17 @@ private void setScanParams(TFileRangeDesc rangeDesc, MaxComputeSplit maxComputeS rangeDesc.setSize(maxComputeSplit.getLength()); } - void createTableBatchReadSession() throws UserException { - List requiredPartitionColumns = new ArrayList<>(); - List orderedRequiredDataColumns = new ArrayList<>(); + private void createRequiredColumns() { Set requiredSlots = desc.getSlots().stream().map(e -> e.getColumn().getName()).collect(Collectors.toSet()); Set partitionColumns = table.getPartitionColumns().stream().map(Column::getName).collect(Collectors.toSet()); + requiredPartitionColumns.clear(); + orderedRequiredDataColumns.clear(); + for (Column column : table.getColumns()) { String columnName = column.getName(); if (!requiredSlots.contains(columnName)) { @@ -138,31 +174,118 @@ void createTableBatchReadSession() throws UserException { orderedRequiredDataColumns.add(columnName); } } + } + /** + * For no partition table: request requiredPartitionSpecs is empty + * For partition table: if requiredPartitionSpecs is empty, get all partition data. + */ + TableBatchReadSession createTableBatchReadSession(List requiredPartitionSpecs) throws IOException { + MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + readTimeout = mcCatalog.getReadTimeout(); + connectTimeout = mcCatalog.getConnectTimeout(); + retryTimes = mcCatalog.getRetryTimes(); + + TableReadSessionBuilder scanBuilder = new TableReadSessionBuilder(); + return scanBuilder.identifier(TableIdentifier.of(table.getDbName(), table.getName())) + .withSettings(mcCatalog.getSettings()) + .withSplitOptions(mcCatalog.getSplitOption()) + .requiredPartitionColumns(requiredPartitionColumns) + .requiredDataColumns(orderedRequiredDataColumns) + .withFilterPredicate(filterPredicate) + .requiredPartitions(requiredPartitionSpecs) + .withArrowOptions( + ArrowOptions.newBuilder() + .withDatetimeUnit(TimestampUnit.MILLI) + .withTimestampUnit(TimestampUnit.NANO) + .build() + ).buildBatchReadSession(); + } - MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + @Override + public boolean isBatchMode() { + if (table.getPartitionColumns().isEmpty()) { + return false; + } - try { - TableReadSessionBuilder scanBuilder = new TableReadSessionBuilder(); - tableBatchReadSession = - scanBuilder.identifier(TableIdentifier.of(table.getDbName(), table.getName())) - .withSettings(mcCatalog.getSettings()) - .withSplitOptions(mcCatalog.getSplitOption()) - .requiredPartitionColumns(requiredPartitionColumns) - .requiredDataColumns(orderedRequiredDataColumns) - .withArrowOptions( - ArrowOptions.newBuilder() - .withDatetimeUnit(TimestampUnit.MILLI) - .withTimestampUnit(TimestampUnit.NANO) - .build() - ) - .withFilterPredicate(filterPredicate) - .buildBatchReadSession(); - } catch (java.io.IOException e) { - throw new RuntimeException(e); + com.aliyun.odps.Table odpsTable = table.getOdpsTable(); + if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { + return false; } + int numPartitions = ConnectContext.get().getSessionVariable().getNumPartitionsInBatchMode(); + return numPartitions > 0 + && selectedPartitions != SelectedPartitions.NOT_PRUNED + && selectedPartitions.selectedPartitions.size() >= numPartitions; + } + + @Override + public int numApproximateSplits() { + return selectedPartitions.selectedPartitions.size(); + } + + @Override + public void startSplit() { + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); + + if (selectedPartitions.selectedPartitions.isEmpty()) { + //no need read any partition data. + return; + } + + createRequiredColumns(); + List requiredPartitionSpecs = new ArrayList<>(); + selectedPartitions.selectedPartitions.forEach( + (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) + ); + + + int batchNumPartitions = ConnectContext.get().getSessionVariable().getNumPartitionsInBatchMode(); + + Executor scheduleExecutor = Env.getCurrentEnv().getExtMetaCacheMgr().getScheduleExecutor(); + AtomicReference batchException = new AtomicReference<>(null); + AtomicInteger numFinishedPartitions = new AtomicInteger(0); + + CompletableFuture.runAsync(() -> { + for (int beginIndex = 0; beginIndex < requiredPartitionSpecs.size(); beginIndex += batchNumPartitions) { + int endIndex = Math.min(beginIndex + batchNumPartitions, requiredPartitionSpecs.size()); + if (batchException.get() != null || splitAssignment.isStop()) { + break; + } + List requiredBatchPartitionSpecs = requiredPartitionSpecs.subList(beginIndex, endIndex); + int curBatchSize = endIndex - beginIndex; + + try { + CompletableFuture.runAsync(() -> { + try { + TableBatchReadSession tableBatchReadSession = + createTableBatchReadSession(requiredBatchPartitionSpecs); + List batchSplit = getSplitByTableSession(tableBatchReadSession); + + splitAssignment.addToQueue(batchSplit); + } catch (IOException e) { + batchException.set(new UserException(e.getMessage(), e)); + } finally { + if (batchException.get() != null) { + splitAssignment.setException(batchException.get()); + } + + if (numFinishedPartitions.addAndGet(curBatchSize) == requiredPartitionSpecs.size()) { + splitAssignment.finishSchedule(); + } + } + }, scheduleExecutor); + } catch (Exception e) { + batchException.set(new UserException(e.getMessage(), e)); + } + + if (batchException.get() != null) { + splitAssignment.setException(batchException.get()); + } + } + }); } @Override @@ -423,60 +546,84 @@ protected Map getLocationProperties() throws UserException { return new HashMap<>(); } - @Override - public List getSplits() throws UserException { + List getSplitByTableSession(TableBatchReadSession tableBatchReadSession) throws java.io.IOException { List result = new ArrayList<>(); - com.aliyun.odps.Table odpsTable = table.getOdpsTable(); - if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { - return result; - } - createTableBatchReadSession(); + String scanSessionSerialize = serializeSession(tableBatchReadSession); + InputSplitAssigner assigner = tableBatchReadSession.getInputSplitAssigner(); + long modificationTime = table.getOdpsTable().getLastDataModifiedTime().getTime(); - try { - String scanSessionSerialize = serializeSession(tableBatchReadSession); - InputSplitAssigner assigner = tableBatchReadSession.getInputSplitAssigner(); - long modificationTime = table.getOdpsTable().getLastDataModifiedTime().getTime(); + MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); - MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + if (mcCatalog.getSplitStrategy().equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { - if (mcCatalog.getSplitStrategy().equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { + for (com.aliyun.odps.table.read.split.InputSplit split : assigner.getAllSplits()) { + MaxComputeSplit maxComputeSplit = + new MaxComputeSplit(BYTE_SIZE_PATH, + ((IndexedInputSplit) split).getSplitIndex(), -1, + mcCatalog.getSplitByteSize(), + modificationTime, null, + Collections.emptyList()); - for (com.aliyun.odps.table.read.split.InputSplit split : assigner.getAllSplits()) { - MaxComputeSplit maxComputeSplit = - new MaxComputeSplit(BYTE_SIZE_PATH, - ((IndexedInputSplit) split).getSplitIndex(), -1, - mcCatalog.getSplitByteSize(), - modificationTime, null, - Collections.emptyList()); + maxComputeSplit.scanSerialize = scanSessionSerialize; + maxComputeSplit.splitType = SplitType.BYTE_SIZE; + maxComputeSplit.sessionId = split.getSessionId(); - maxComputeSplit.scanSerialize = scanSessionSerialize; - maxComputeSplit.splitType = SplitType.BYTE_SIZE; - maxComputeSplit.sessionId = split.getSessionId(); + result.add(maxComputeSplit); + } + } else { + long totalRowCount = assigner.getTotalRowCount(); - result.add(maxComputeSplit); - } - } else { - long totalRowCount = assigner.getTotalRowCount(); + long recordsPerSplit = mcCatalog.getSplitRowCount(); + for (long offset = 0; offset < totalRowCount; offset += recordsPerSplit) { + recordsPerSplit = Math.min(recordsPerSplit, totalRowCount - offset); + com.aliyun.odps.table.read.split.InputSplit split = + assigner.getSplitByRowOffset(offset, recordsPerSplit); - long recordsPerSplit = mcCatalog.getSplitRowCount(); - for (long offset = 0; offset < totalRowCount; offset += recordsPerSplit) { - recordsPerSplit = Math.min(recordsPerSplit, totalRowCount - offset); - com.aliyun.odps.table.read.split.InputSplit split = - assigner.getSplitByRowOffset(offset, recordsPerSplit); + MaxComputeSplit maxComputeSplit = + new MaxComputeSplit(ROW_OFFSET_PATH, + offset, recordsPerSplit, totalRowCount, modificationTime, null, + Collections.emptyList()); - MaxComputeSplit maxComputeSplit = - new MaxComputeSplit(ROW_OFFSET_PATH, - offset, recordsPerSplit, totalRowCount, modificationTime, null, - Collections.emptyList()); + maxComputeSplit.scanSerialize = scanSessionSerialize; + maxComputeSplit.splitType = SplitType.ROW_OFFSET; + maxComputeSplit.sessionId = split.getSessionId(); - maxComputeSplit.scanSerialize = scanSessionSerialize; - maxComputeSplit.splitType = SplitType.ROW_OFFSET; - maxComputeSplit.sessionId = split.getSessionId(); + result.add(maxComputeSplit); + } + } + return result; + } - result.add(maxComputeSplit); - } + + @Override + public List getSplits() throws UserException { + List result = new ArrayList<>(); + com.aliyun.odps.Table odpsTable = table.getOdpsTable(); + if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { + return result; + } + + createRequiredColumns(); + + List requiredPartitionSpecs = new ArrayList<>(); + //if requiredPartitionSpecs is empty, get all partition data. + if (!table.getPartitionColumns().isEmpty() && selectedPartitions != SelectedPartitions.NOT_PRUNED) { + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); + + if (selectedPartitions.selectedPartitions.isEmpty()) { + //no need read any partition data. + return result; } + selectedPartitions.selectedPartitions.forEach( + (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) + ); + } + + try { + TableBatchReadSession tableBatchReadSession = createTableBatchReadSession(requiredPartitionSpecs); + result = getSplitByTableSession(tableBatchReadSession); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java index 4a2757f918f294..50166fe8305113 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOperations.java @@ -17,9 +17,9 @@ package org.apache.doris.datasource.operations; +import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HiveMetadataOps; -import org.apache.doris.datasource.iceberg.IcebergExternalCatalog; import org.apache.doris.datasource.iceberg.IcebergMetadataOps; import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; @@ -34,7 +34,7 @@ public static HiveMetadataOps newHiveMetadataOps(HiveConf hiveConf, JdbcClientCo return new HiveMetadataOps(hiveConf, jdbcClientConfig, catalog); } - public static IcebergMetadataOps newIcebergMetadataOps(IcebergExternalCatalog dorisCatalog, Catalog catalog) { + public static IcebergMetadataOps newIcebergMetadataOps(ExternalCatalog dorisCatalog, Catalog catalog) { return new IcebergMetadataOps(dorisCatalog, catalog); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java index 0333124b35294c..e5ed129c679ffe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/operations/ExternalMetadataOps.java @@ -91,6 +91,10 @@ public interface ExternalMetadataOps { boolean databaseExist(String dbName); + default Object loadTable(String dbName, String tblName) { + throw new UnsupportedOperationException("Load table is not supported."); + } + /** * close the connection, eg, to hms */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java index 5a9e6feb5ad5b8..eb25336ab0b033 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalCatalog.java @@ -19,7 +19,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopUGI; +import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.InitCatalogLog; @@ -40,6 +40,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.options.Options; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -53,6 +54,7 @@ public abstract class PaimonExternalCatalog extends ExternalCatalog { protected String catalogType; protected Catalog catalog; protected AuthenticationConfig authConf; + protected HadoopAuthenticator hadoopAuthenticator; private static final List REQUIRED_PROPERTIES = ImmutableList.of( PaimonProperties.WAREHOUSE @@ -71,9 +73,8 @@ protected void initLocalObjectsImpl() { for (Map.Entry propEntry : this.catalogProperty.getHadoopProperties().entrySet()) { conf.set(propEntry.getKey(), propEntry.getValue()); } - authConf = AuthenticationConfig.getKerberosConfig(conf, - AuthenticationConfig.HADOOP_KERBEROS_PRINCIPAL, - AuthenticationConfig.HADOOP_KERBEROS_KEYTAB); + authConf = AuthenticationConfig.getKerberosConfig(conf); + hadoopAuthenticator = HadoopAuthenticator.getHadoopAuthenticator(authConf); } public String getCatalogType() { @@ -82,40 +83,57 @@ public String getCatalogType() { } protected List listDatabaseNames() { - return HadoopUGI.ugiDoAs(authConf, () -> new ArrayList<>(catalog.listDatabases())); + try { + return hadoopAuthenticator.doAs(() -> new ArrayList<>(catalog.listDatabases())); + } catch (IOException e) { + throw new RuntimeException("Failed to list databases names, catalog name: " + getName(), e); + } } @Override public boolean tableExist(SessionContext ctx, String dbName, String tblName) { makeSureInitialized(); - return HadoopUGI.ugiDoAs(authConf, () -> catalog.tableExists(Identifier.create(dbName, tblName))); + try { + return hadoopAuthenticator.doAs(() -> catalog.tableExists(Identifier.create(dbName, tblName))); + } catch (IOException e) { + throw new RuntimeException("Failed to check table existence, catalog name: " + getName(), e); + } } @Override public List listTableNames(SessionContext ctx, String dbName) { makeSureInitialized(); - return HadoopUGI.ugiDoAs(authConf, () -> { - List tableNames = null; - try { - tableNames = catalog.listTables(dbName); - } catch (Catalog.DatabaseNotExistException e) { - LOG.warn("DatabaseNotExistException", e); - } - return tableNames; - }); + try { + return hadoopAuthenticator.doAs(() -> { + List tableNames = null; + try { + tableNames = catalog.listTables(dbName); + } catch (Catalog.DatabaseNotExistException e) { + LOG.warn("DatabaseNotExistException", e); + } + return tableNames; + }); + } catch (IOException e) { + throw new RuntimeException("Failed to list table names, catalog name: " + getName(), e); + } } public org.apache.paimon.table.Table getPaimonTable(String dbName, String tblName) { makeSureInitialized(); - return HadoopUGI.ugiDoAs(authConf, () -> { - org.apache.paimon.table.Table table = null; - try { - table = catalog.getTable(Identifier.create(dbName, tblName)); - } catch (Catalog.TableNotExistException e) { - LOG.warn("TableNotExistException", e); - } - return table; - }); + try { + return hadoopAuthenticator.doAs(() -> { + org.apache.paimon.table.Table table = null; + try { + table = catalog.getTable(Identifier.create(dbName, tblName)); + } catch (Catalog.TableNotExistException e) { + LOG.warn("TableNotExistException", e); + } + return table; + }); + } catch (IOException e) { + throw new RuntimeException("Failed to get Paimon table, catalog name: " + getName() + ", db: " + + dbName + ", table: " + tblName, e); + } } protected String getPaimonCatalogType(String catalogType) { @@ -127,15 +145,19 @@ protected String getPaimonCatalogType(String catalogType) { } protected Catalog createCatalog() { - return HadoopUGI.ugiDoAs(authConf, () -> { - Options options = new Options(); - Map paimonOptionsMap = getPaimonOptionsMap(); - for (Map.Entry kv : paimonOptionsMap.entrySet()) { - options.set(kv.getKey(), kv.getValue()); - } - CatalogContext context = CatalogContext.create(options, getConfiguration()); - return createCatalogImpl(context); - }); + try { + return hadoopAuthenticator.doAs(() -> { + Options options = new Options(); + Map paimonOptionsMap = getPaimonOptionsMap(); + for (Map.Entry kv : paimonOptionsMap.entrySet()) { + options.set(kv.getKey(), kv.getValue()); + } + CatalogContext context = CatalogContext.create(options, getConfiguration()); + return createCatalogImpl(context); + }); + } catch (IOException e) { + throw new RuntimeException("Failed to create catalog, catalog name: " + getName(), e); + } } protected Catalog createCatalogImpl(CatalogContext context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java index 59f51c8425c7f2..5009ec3c9049fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/source/PaimonScanNode.java @@ -105,9 +105,9 @@ public String toString() { private String serializedTable; public PaimonScanNode(PlanNodeId id, - TupleDescriptor desc, - boolean needCheckColumnPriv, - SessionVariable sessionVariable) { + TupleDescriptor desc, + boolean needCheckColumnPriv, + SessionVariable sessionVariable) { super(id, desc, "PAIMON_SCAN_NODE", StatisticalType.PAIMON_SCAN_NODE, needCheckColumnPriv); this.sessionVariable = sessionVariable; } @@ -127,8 +127,7 @@ protected void convertPredicate() { predicates = paimonPredicateConverter.convertToPaimonExpr(conjuncts); } - private static final Base64.Encoder BASE64_ENCODER = - java.util.Base64.getUrlEncoder().withoutPadding(); + private static final Base64.Encoder BASE64_ENCODER = java.util.Base64.getUrlEncoder().withoutPadding(); public static String encodeObjectToString(T t) { try { @@ -156,11 +155,24 @@ private void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) tableFormatFileDesc.setTableFormatType(paimonSplit.getTableFormatType().value()); TPaimonFileDesc fileDesc = new TPaimonFileDesc(); org.apache.paimon.table.source.Split split = paimonSplit.getSplit(); + + String fileFormat = getFileFormat(paimonSplit.getPathString()); if (split != null) { // use jni reader + rangeDesc.setFormatType(TFileFormatType.FORMAT_JNI); fileDesc.setPaimonSplit(encodeObjectToString(split)); + } else { + // use native reader + if (fileFormat.equals("orc")) { + rangeDesc.setFormatType(TFileFormatType.FORMAT_ORC); + } else if (fileFormat.equals("parquet")) { + rangeDesc.setFormatType(TFileFormatType.FORMAT_PARQUET); + } else { + throw new RuntimeException("Unsupported file format: " + fileFormat); + } } - fileDesc.setFileFormat(getFileFormat(paimonSplit.getPathString())); + + fileDesc.setFileFormat(fileFormat); fileDesc.setPaimonPredicate(encodeObjectToString(predicates)); fileDesc.setPaimonColumnNames(source.getDesc().getSlots().stream().map(slot -> slot.getColumn().getName()) .collect(Collectors.joining(","))); @@ -171,7 +183,6 @@ private void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) fileDesc.setDbId(((PaimonExternalTable) source.getTargetTable()).getDbId()); fileDesc.setTblId(source.getTargetTable().getId()); fileDesc.setLastUpdateTime(source.getTargetTable().getUpdateTime()); - fileDesc.setPaimonTable(encodeObjectToString(source.getPaimonTable())); // The hadoop conf should be same with PaimonExternalCatalog.createCatalog()#getConfiguration() fileDesc.setHadoopConf(source.getCatalog().getCatalogProperty().getHadoopProperties()); Optional optDeletionFile = paimonSplit.getDeletionFile(); @@ -190,8 +201,8 @@ private void setPaimonParams(TFileRangeDesc rangeDesc, PaimonSplit paimonSplit) @Override public List getSplits() throws UserException { boolean forceJniScanner = sessionVariable.isForceJniScanner(); - SessionVariable.IgnoreSplitType ignoreSplitType = - SessionVariable.IgnoreSplitType.valueOf(sessionVariable.getIgnoreSplitType()); + SessionVariable.IgnoreSplitType ignoreSplitType = SessionVariable.IgnoreSplitType + .valueOf(sessionVariable.getIgnoreSplitType()); List splits = new ArrayList<>(); int[] projected = desc.getSlots().stream().mapToInt( slot -> (source.getPaimonTable().rowType().getFieldNames().indexOf(slot.getColumn().getName()))) @@ -288,7 +299,8 @@ public List getSplits() throws UserException { } this.selectedPartitionNum = selectedPartitionValues.size(); // TODO: get total partition number - // We should set fileSplitSize at the end because fileSplitSize may be modified in splitFile. + // We should set fileSplitSize at the end because fileSplitSize may be modified + // in splitFile. splits.forEach(s -> s.setTargetSplitSize(fileSplitSize)); return splits; } @@ -318,8 +330,9 @@ public TFileFormatType getFileFormatType() throws DdlException, MetaNotFoundExce @Override public List getPathPartitionKeys() throws DdlException, MetaNotFoundException { - // return new ArrayList<>(source.getPaimonTable().partitionKeys()); - //Paymon is not aware of partitions and bypasses some existing logic by returning an empty list + // return new ArrayList<>(source.getPaimonTable().partitionKeys()); + // Paymon is not aware of partitions and bypasses some existing logic by + // returning an empty list return new ArrayList<>(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java index 20a77574fc7820..3957ffc91bb4b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java @@ -56,6 +56,21 @@ public class MCProperties extends BaseProperties { public static final String SPLIT_ROW_COUNT = "mc.split_row_count"; public static final String DEFAULT_SPLIT_ROW_COUNT = "1048576"; // 256 * 4096 + public static final String CONNECT_TIMEOUT = "mc.connect_timeout"; + public static final String READ_TIMEOUT = "mc.read_timeout"; + public static final String RETRY_COUNT = "mc.retry_count"; + + public static final String DEFAULT_CONNECT_TIMEOUT = "10"; // 10s + public static final String DEFAULT_READ_TIMEOUT = "120"; // 120s + public static final String DEFAULT_RETRY_COUNT = "4"; // 4 times + + //withCrossPartition(true): + // Very friendly to scenarios where there are many partitions but each partition is very small. + //withCrossPartition(false): + // Very debug friendly. + public static final String SPLIT_CROSS_PARTITION = "mc.split_cross_partition"; + public static final String DEFAULT_SPLIT_CROSS_PARTITION = "true"; + public static CloudCredential getCredential(Map props) { return getCloudCredential(props, ACCESS_KEY, SECRET_KEY, SESSION_TOKEN); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java index 475ee5ace1ea0d..87c4c4cfa90a68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetLogFileAction.java @@ -32,6 +32,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Map; import java.util.Set; import javax.servlet.http.HttpServletRequest; @@ -51,6 +53,23 @@ */ @RestController public class GetLogFileAction extends RestBaseController { + /** + * This method fetches internal logs via HTTP, which is no longer recommended and will + * be deprecated in future versions. + *

+ * Using HTTP to fetch logs introduces serious security and performance issues: + * - **Security Risks**: Log content may expose sensitive information, allowing attackers to exploit the exposed + * HTTP endpoints. + * - **Performance Problems**: Frequent HTTP requests can cause significant system load, affecting the + * responsiveness and stability of the application. + *

+ * It is strongly advised not to use this approach for accessing logs. Any new requirements should be + * handled using more secure, reliable, and efficient methods such as log aggregation tools (e.g., ELK, Splunk) + * or dedicated internal APIs. + *

+ * **Note**: No new HTTP endpoints or types for log access will be accepted. + * Any further attempts to extend this HTTP-based log retrieval method will not be supported. + */ private final Set logFileTypes = Sets.newHashSet("fe.audit.log"); @RequestMapping(path = "/api/get_log_file", method = {RequestMethod.GET, RequestMethod.HEAD}) @@ -79,7 +98,13 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) String fileInfos = getFileInfos(logType); response.setHeader("file_infos", fileInfos); return ResponseEntityBuilder.ok(); - } else if (method.equals(RequestMethod.GET.name())) { + } + if (method.equals(RequestMethod.GET.name())) { + try { + checkAuditLogFileName(logFile); + } catch (SecurityException e) { + return ResponseEntityBuilder.internalError(e.getMessage()); + } File log = getLogFile(logType, logFile); if (!log.exists() || !log.isFile()) { return ResponseEntityBuilder.okWithCommonError("Log file not exist: " + log.getName()); @@ -97,6 +122,17 @@ public Object execute(HttpServletRequest request, HttpServletResponse response) return ResponseEntityBuilder.ok(); } + private void checkAuditLogFileName(String logFile) { + if (!logFile.matches("^[a-zA-Z0-9._-]+$")) { + throw new SecurityException("Invalid file name"); + } + Path normalizedPath = Paths.get(Config.audit_log_dir).resolve(logFile).normalize(); + // check path is valid or not + if (!normalizedPath.startsWith(Config.audit_log_dir)) { + throw new SecurityException("Invalid file path: Access outside of permitted directory"); + } + } + private String getFileInfos(String logType) { Map fileInfos = Maps.newTreeMap(); if (logType.equals("fe.audit.log")) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.java b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.java index 3d8f9ed15349ae..ac67715d2ca18e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.java @@ -34,4 +34,9 @@ public static EventFactory> factory return ExecuteTaskEvent::new; } + public void clear() { + this.task = null; + this.jobConfig = null; + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java index 6ca2924c593bc1..9fb9d94e8df70f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java @@ -19,7 +19,6 @@ import com.lmax.disruptor.EventFactory; import com.lmax.disruptor.EventTranslatorVararg; -import com.lmax.disruptor.RingBuffer; import com.lmax.disruptor.WaitStrategy; import com.lmax.disruptor.WorkHandler; import com.lmax.disruptor.dsl.Disruptor; @@ -28,6 +27,7 @@ import org.apache.logging.log4j.Logger; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; /** * Utility class for creating and managing a Disruptor instance. @@ -73,11 +73,42 @@ public void start() { */ public boolean publishEvent(Object... args) { try { - RingBuffer ringBuffer = disruptor.getRingBuffer(); - return ringBuffer.tryPublishEvent(eventTranslator, args); + // Set the timeout to 1 second, converted to nanoseconds for precision + long timeoutInNanos = TimeUnit.SECONDS.toNanos(1); // Timeout set to 1 second + long startTime = System.nanoTime(); // Record the start time + + // Loop until the timeout is reached + while (System.nanoTime() - startTime < timeoutInNanos) { + // Check if there is enough remaining capacity in the ring buffer + // Adjusting to check if the required capacity is available (instead of hardcoding 1) + if (disruptor.getRingBuffer().remainingCapacity() > 1) { + // Publish the event if there is enough capacity + disruptor.getRingBuffer().publishEvent(eventTranslator, args); + if (LOG.isDebugEnabled()) { + LOG.debug("publishEvent success,the remaining buffer size is {}", + disruptor.getRingBuffer().remainingCapacity()); + } + return true; + } + + // Wait for a short period before retrying + try { + Thread.sleep(10); // Adjust the wait time as needed (maybe increase if not high-frequency) + } catch (InterruptedException e) { + // Log the exception and return false if interrupted + Thread.currentThread().interrupt(); // Restore interrupt status + LOG.warn("Thread interrupted while waiting to publish event", e); + return false; + } + } + + // Timeout reached without publishing the event + LOG.warn("Failed to publish event within the specified timeout (1 second)." + + "Queue may be full. the remaining buffer size is {}", + disruptor.getRingBuffer().remainingCapacity()); } catch (Exception e) { - LOG.warn("Failed to publish event", e); - // Handle the exception, e.g., retry or alert + // Catching general exceptions to handle unexpected errors + LOG.warn("Failed to publish event due to an unexpected error", e); } return false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.java index befa8cc35fcbcc..cdfe7c0fe08f63 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.java @@ -36,35 +36,23 @@ public class DefaultTaskExecutorHandler implements WorkH @Override public void onEvent(ExecuteTaskEvent executeTaskEvent) { - T task = executeTaskEvent.getTask(); - if (null == task) { - log.warn("task is null, ignore,maybe task has been canceled"); - return; - } - if (task.isCancelled()) { - log.info("task is canceled, ignore. task id is {}", task.getTaskId()); - return; - } - log.info("start to execute task, task id is {}", task.getTaskId()); - try { - task.runTask(); - } catch (Exception e) { - //if task.onFail() throw exception, we will catch it here - log.warn("task before error, task id is {}", task.getTaskId(), e); - } - //todo we need discuss whether we need to use semaphore to control the concurrent task num - /* Semaphore semaphore = null; - // get token try { - int maxConcurrentTaskNum = executeTaskEvent.getJobConfig().getMaxConcurrentTaskNum(); - semaphore = TaskTokenManager.tryAcquire(task.getJobId(), maxConcurrentTaskNum); + T task = executeTaskEvent.getTask(); + if (null == task) { + log.warn("task is null, ignore,maybe task has been canceled"); + return; + } + if (task.isCancelled()) { + log.info("task is canceled, ignore. task id is {}", task.getTaskId()); + return; + } + log.info("start to execute task, task id is {}", task.getTaskId()); task.runTask(); } catch (Exception e) { - task.onFail(); - log.error("execute task error, task id is {}", task.getTaskId(), e); + log.error("execute task error, task id is {}", executeTaskEvent.getTask().getTaskId(), e); } finally { - if (null != semaphore) { - semaphore.release(); - }*/ + executeTaskEvent.clear(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java index d93393aa0ef89f..b8f726c4a0c76f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -21,7 +21,6 @@ import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.TaskType; -import org.apache.doris.job.disruptor.TaskDisruptor; import org.apache.doris.job.disruptor.TimerJobEvent; import org.apache.doris.job.task.AbstractTask; @@ -40,9 +39,9 @@ @Log4j2 public class DispatchTaskHandler implements WorkHandler> { - private final Map> disruptorMap; + private final Map disruptorMap; - public DispatchTaskHandler(Map> disruptorMap) { + public DispatchTaskHandler(Map disruptorMap) { this.disruptorMap = disruptorMap; } @@ -66,7 +65,7 @@ public void onEvent(TimerJobEvent event) { } JobType jobType = event.getJob().getJobType(); for (AbstractTask task : tasks) { - if (!disruptorMap.get(jobType).publishEvent(task, event.getJob().getJobConfig())) { + if (!disruptorMap.get(jobType).addTask(task)) { task.cancel(); continue; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TaskProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TaskProcessor.java new file mode 100644 index 00000000000000..d9d3f25dcd8a80 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TaskProcessor.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.job.executor; + +import org.apache.doris.job.task.AbstractTask; + +import lombok.extern.log4j.Log4j2; + +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +@Log4j2 +public class TaskProcessor { + private ExecutorService executor; + + public TaskProcessor(int numberOfThreads, int queueSize, ThreadFactory threadFactory) { + this.executor = new ThreadPoolExecutor( + numberOfThreads, + numberOfThreads, + 0L, TimeUnit.MILLISECONDS, + new ArrayBlockingQueue<>(queueSize), + threadFactory, + new ThreadPoolExecutor.AbortPolicy() + ); + } + + public boolean addTask(AbstractTask task) { + try { + executor.execute(() -> runTask(task)); + log.info("Add task to executor, task id: {}", task.getTaskId()); + return true; + } catch (RejectedExecutionException e) { + log.warn("Failed to add task to executor, task id: {}", task.getTaskId(), e); + return false; + } + } + + public void shutdown() { + log.info("Shutting down executor service..."); + executor.shutdown(); + try { + if (!executor.awaitTermination(60, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException e) { + executor.shutdownNow(); + Thread.currentThread().interrupt(); + } + log.info("Executor service shut down successfully."); + } + + private void runTask(AbstractTask task) { + try { + if (task == null) { + log.warn("Task is null, ignore. Maybe it has been canceled."); + return; + } + if (task.isCancelled()) { + log.info("Task is canceled, ignore. Task id: {}", task.getTaskId()); + return; + } + log.info("Start to execute task, task id: {}", task.getTaskId()); + task.runTask(); + } catch (Exception e) { + log.warn("Execute task error, task id: {}", task.getTaskId(), e); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java index 65a9cf2e091164..4269fa0d8f3c52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java @@ -48,6 +48,8 @@ public void run(Timeout timeout) { log.warn("dispatch timer job failed, queue maybe full. job id is {}, job name is {}", this.job.getJobId(), this.job.getJobName() + getMsgWhenExecuteQueueFull()); } + log.info("dispatch timer job success, job id is {}, job name is {}", this.job.getJobId(), + this.job.getJobName()); } catch (Exception e) { log.warn("dispatch timer job error, task id is {}", this.job.getJobId(), e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java index cc82b59a36a36f..e77dfbadcb3742 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java @@ -22,13 +22,10 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.common.JobType; -import org.apache.doris.job.disruptor.ExecuteTaskEvent; import org.apache.doris.job.disruptor.TaskDisruptor; import org.apache.doris.job.disruptor.TimerJobEvent; -import org.apache.doris.job.executor.DefaultTaskExecutorHandler; import org.apache.doris.job.executor.DispatchTaskHandler; -import org.apache.doris.job.extensions.insert.InsertTask; -import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.job.executor.TaskProcessor; import org.apache.doris.job.task.AbstractTask; import com.lmax.disruptor.EventFactory; @@ -44,7 +41,7 @@ public class TaskDisruptorGroupManager { - private final Map> disruptorMap = new EnumMap<>(JobType.class); + private final Map disruptorMap = new EnumMap<>(JobType.class); @Getter private TaskDisruptor> dispatchDisruptor; @@ -92,44 +89,27 @@ private void registerDispatchDisruptor() { } private void registerInsertDisruptor() { - EventFactory> insertEventFactory = ExecuteTaskEvent.factory(); ThreadFactory insertTaskThreadFactory = new CustomThreadFactory("insert-task-execute"); - WorkHandler[] insertTaskExecutorHandlers = new WorkHandler[DISPATCH_INSERT_THREAD_NUM]; - for (int i = 0; i < DISPATCH_INSERT_THREAD_NUM; i++) { - insertTaskExecutorHandlers[i] = new DefaultTaskExecutorHandler(); - } - EventTranslatorVararg> eventTranslator = - (event, sequence, args) -> { - event.setTask((InsertTask) args[0]); - event.setJobConfig((JobExecutionConfiguration) args[1]); - }; - TaskDisruptor insertDisruptor = new TaskDisruptor<>(insertEventFactory, DISPATCH_INSERT_TASK_QUEUE_SIZE, - insertTaskThreadFactory, new LiteTimeoutBlockingWaitStrategy(10, TimeUnit.MILLISECONDS), - insertTaskExecutorHandlers, eventTranslator); - disruptorMap.put(JobType.INSERT, insertDisruptor); + + + TaskProcessor insertTaskProcessor = new TaskProcessor(DISPATCH_INSERT_THREAD_NUM, + DISPATCH_INSERT_TASK_QUEUE_SIZE, insertTaskThreadFactory); + disruptorMap.put(JobType.INSERT, insertTaskProcessor); } private void registerMTMVDisruptor() { - EventFactory> mtmvEventFactory = ExecuteTaskEvent.factory(); + ThreadFactory mtmvTaskThreadFactory = new CustomThreadFactory("mtmv-task-execute"); - WorkHandler[] insertTaskExecutorHandlers = new WorkHandler[DISPATCH_MTMV_THREAD_NUM]; - for (int i = 0; i < DISPATCH_MTMV_THREAD_NUM; i++) { - insertTaskExecutorHandlers[i] = new DefaultTaskExecutorHandler(); - } - EventTranslatorVararg> eventTranslator = - (event, sequence, args) -> { - event.setTask((MTMVTask) args[0]); - event.setJobConfig((JobExecutionConfiguration) args[1]); - }; - TaskDisruptor mtmvDisruptor = new TaskDisruptor<>(mtmvEventFactory, DISPATCH_MTMV_TASK_QUEUE_SIZE, - mtmvTaskThreadFactory, new LiteTimeoutBlockingWaitStrategy(10, TimeUnit.MILLISECONDS), - insertTaskExecutorHandlers, eventTranslator); - disruptorMap.put(JobType.MV, mtmvDisruptor); + TaskProcessor mtmvTaskProcessor = new TaskProcessor(DISPATCH_MTMV_THREAD_NUM, + DISPATCH_MTMV_TASK_QUEUE_SIZE, mtmvTaskThreadFactory); + disruptorMap.put(JobType.MV, mtmvTaskProcessor); } public boolean dispatchInstantTask(AbstractTask task, JobType jobType, JobExecutionConfiguration jobExecutionConfiguration) { - return disruptorMap.get(jobType).publishEvent(task, jobExecutionConfiguration); + + + return disruptorMap.get(jobType).addTask(task); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java index ea0c263a5ee1c3..921f333791cb44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -146,6 +146,11 @@ public void close() throws IOException { private void cycleTimerJobScheduler(T job, long startTimeWindowMs) { List delaySeconds = job.getJobConfig().getTriggerDelayTimes(System.currentTimeMillis(), startTimeWindowMs, latestBatchSchedulerTimerTaskTimeMs); + if (CollectionUtils.isEmpty(delaySeconds)) { + log.info("skip job {} scheduler timer job, delay seconds is empty", job.getJobName()); + return; + } + log.info("job {} scheduler timer job, delay seconds size is {}", job.getJobName(), delaySeconds.size()); if (CollectionUtils.isNotEmpty(delaySeconds)) { delaySeconds.forEach(delaySecond -> { TimerJobSchedulerTask timerJobSchedulerTask = new TimerJobSchedulerTask<>(timerJobDisruptor, job); @@ -168,7 +173,9 @@ public void schedulerInstantJob(T job, TaskType taskType, C context) throws JobE for (AbstractTask task : tasks) { if (!taskDisruptorGroupManager.dispatchInstantTask(task, job.getJobType(), job.getJobConfig())) { - throw new JobException(job.formatMsgWhenExecuteQueueFull(task.getTaskId())); + String errorMsg = job.formatMsgWhenExecuteQueueFull(task.getTaskId()); + task.onFail(errorMsg); + throw new JobException(errorMsg); } log.info("dispatch instant job, job id is {}, job name is {}, task id is {}", job.getJobId(), @@ -186,6 +193,8 @@ private void executeTimerJobIdsWithinLastTenMinutesWindow() { this.latestBatchSchedulerTimerTaskTimeMs = System.currentTimeMillis(); } this.latestBatchSchedulerTimerTaskTimeMs += BATCH_SCHEDULER_INTERVAL_MILLI_SECONDS; + log.info("execute timer job ids within last ten minutes window, last time window is {}", + TimeUtils.longToTimeString(lastTimeWindowMs)); if (jobMap.isEmpty()) { return; } @@ -207,6 +216,7 @@ private void clearEndJob(T job) { } try { Env.getCurrentEnv().getJobManager().unregisterJob(job.getJobId()); + log.info("clear finish job, job id is {}, job name is {}", job.getJobId(), job.getJobName()); } catch (JobException e) { log.error("clear finish job error, job id is {}", job.getJobId(), e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java index 49ebbfe7dcddb0..238b40a7d41679 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java @@ -106,26 +106,24 @@ public void addExportJobAndRegisterTask(ExportJob job) throws Exception { } } unprotectAddJob(job); - // delete existing files - if (Config.enable_delete_existing_files && Boolean.parseBoolean(job.getDeleteExistingFiles())) { - if (job.getBrokerDesc() == null) { - throw new AnalysisException("Local file system does not support delete existing files"); - } - String fullPath = job.getExportPath(); - BrokerUtil.deleteDirectoryWithFileSystem(fullPath.substring(0, fullPath.lastIndexOf('/') + 1), - job.getBrokerDesc()); - } Env.getCurrentEnv().getEditLog().logExportCreate(job); - // ATTN: Must add task after edit log, otherwise the job may finish before adding job. - job.getCopiedTaskExecutors().forEach(executor -> { - Env.getCurrentEnv().getTransientTaskManager().addMemoryTask(executor); - }); - LOG.info("add export job. {}", job); - } finally { writeUnlock(); } - + // delete existing files + if (Config.enable_delete_existing_files && Boolean.parseBoolean(job.getDeleteExistingFiles())) { + if (job.getBrokerDesc() == null) { + throw new AnalysisException("Local file system does not support delete existing files"); + } + String fullPath = job.getExportPath(); + BrokerUtil.deleteDirectoryWithFileSystem(fullPath.substring(0, fullPath.lastIndexOf('/') + 1), + job.getBrokerDesc()); + } + // ATTN: Must add task after edit log, otherwise the job may finish before adding job. + for (int i = 0; i < job.getCopiedTaskExecutors().size(); i++) { + Env.getCurrentEnv().getTransientTaskManager().addMemoryTask(job.getCopiedTaskExecutors().get(i)); + } + LOG.info("add export job. {}", job); } public void cancelExportJob(CancelExportStmt stmt) throws DdlException, AnalysisException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java index 8afc35411b55f5..d40a6705626c84 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java @@ -106,7 +106,7 @@ private void process() throws UserException, InterruptedException { if (routineLoadTaskInfo.getIsEof()) { RoutineLoadJob routineLoadJob = routineLoadManager.getJob(routineLoadTaskInfo.getJobId()); if (System.currentTimeMillis() - routineLoadTaskInfo.getLastScheduledTime() - < routineLoadJob.getMaxBatchIntervalS()) { + < routineLoadJob.getMaxBatchIntervalS() * 1000) { needScheduleTasksQueue.addLast(routineLoadTaskInfo); return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java index 56061c75b9cee2..d3d7f1ad6ebbf5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java @@ -51,14 +51,18 @@ public class MTMVCache { // The materialized view plan which should be optimized by the same rules to query // and will remove top sink and unused sort private final Plan logicalPlan; - // The original plan of mv def sql + // The original rewritten plan of mv def sql private final Plan originalPlan; + // The analyzed plan of mv def sql, which is used by tableCollector,should not be optimized by rbo + private final Plan analyzedPlan; private final Statistics statistics; private final StructInfo structInfo; - public MTMVCache(Plan logicalPlan, Plan originalPlan, Statistics statistics, StructInfo structInfo) { + public MTMVCache(Plan logicalPlan, Plan originalPlan, Plan analyzedPlan, + Statistics statistics, StructInfo structInfo) { this.logicalPlan = logicalPlan; this.originalPlan = originalPlan; + this.analyzedPlan = analyzedPlan; this.statistics = statistics; this.structInfo = structInfo; } @@ -71,6 +75,10 @@ public Plan getOriginalPlan() { return originalPlan; } + public Plan getAnalyzedPlan() { + return analyzedPlan; + } + public Statistics getStatistics() { return statistics; } @@ -80,13 +88,14 @@ public StructInfo getStructInfo() { } public static MTMVCache from(MTMV mtmv, ConnectContext connectContext, boolean needCost) { - LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(mtmv.getQuerySql()); StatementContext mvSqlStatementContext = new StatementContext(connectContext, new OriginStatement(mtmv.getQuerySql(), 0)); - NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); if (mvSqlStatementContext.getConnectContext().getStatementContext() == null) { mvSqlStatementContext.getConnectContext().setStatementContext(mvSqlStatementContext); } + LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(mtmv.getQuerySql()); + NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); + // Can not convert to table sink, because use the same column from different table when self join // the out slot is wrong if (needCost) { @@ -117,7 +126,7 @@ public Plan visitLogicalResultSink(LogicalResultSink logicalResu Optional structInfoOptional = MaterializationContext.constructStructInfo(mvPlan, originPlan, planner.getCascadesContext(), new BitSet()); - return new MTMVCache(mvPlan, originPlan, needCost + return new MTMVCache(mvPlan, originPlan, planner.getAnalyzedPlan(), needCost ? planner.getCascadesContext().getMemo().getRoot().getStatistics() : null, structInfoOptional.orElseGet(() -> null)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index 8ba022de415006..55ce86e75706c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -172,18 +172,12 @@ public static Map> generateRelatedPartitionDescs(M } public static List getPartitionsIdsByNames(MTMV mtmv, List partitions) throws AnalysisException { - mtmv.readLock(); - try { - List res = Lists.newArrayList(); - for (String partitionName : partitions) { - Partition partition = mtmv.getPartitionOrAnalysisException(partitionName); - res.add(partition.getId()); - } - return res; - } finally { - mtmv.readUnlock(); + List res = Lists.newArrayList(); + for (String partitionName : partitions) { + Partition partition = mtmv.getPartitionOrAnalysisException(partitionName); + res.add(partition.getId()); } - + return res; } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index c0cd47bd5a0f2f..35c06e74d3cc80 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -35,12 +35,10 @@ import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; -import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.visitor.TableCollector; import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.SessionVariable; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; @@ -58,6 +56,12 @@ public static ConnectContext createMTMVContext(MTMV mtmv) { ctx.setCurrentUserIdentity(UserIdentity.ADMIN); ctx.getState().reset(); ctx.setThreadLocalInfo(); + // Debug session variable should be disabled when refreshed + ctx.getSessionVariable().skipDeletePredicate = false; + ctx.getSessionVariable().skipDeleteBitmap = false; + ctx.getSessionVariable().skipDeleteSign = false; + ctx.getSessionVariable().skipStorageEngineMerge = false; + ctx.getSessionVariable().showHiddenColumns = false; ctx.getSessionVariable().allowModifyMaterializedViewData = true; // Disable add default limit rule to avoid refresh data wrong ctx.getSessionVariable().setDisableNereidsRules( @@ -98,31 +102,20 @@ private static void setCatalogAndDb(ConnectContext ctx, MTMV mtmv) { public static MTMVRelation generateMTMVRelation(MTMV mtmv, ConnectContext ctx) { // Should not make table without data to empty relation when analyze the related table, // so add disable rules - SessionVariable sessionVariable = ctx.getSessionVariable(); - Set tempDisableRules = sessionVariable.getDisableNereidsRuleNames(); - sessionVariable.setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); - if (ctx.getStatementContext() != null) { - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - } - Plan plan; - try { - plan = getPlanBySql(mtmv.getQuerySql(), ctx); - } finally { - sessionVariable.setDisableNereidsRules(String.join(",", tempDisableRules)); - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - } - return generateMTMVRelation(plan); + Plan plan = getAnalyzePlanBySql(mtmv.getQuerySql(), ctx); + return generateMTMVRelation(plan, ctx); } - public static MTMVRelation generateMTMVRelation(Plan plan) { - return new MTMVRelation(getBaseTables(plan, true), getBaseTables(plan, false), getBaseViews(plan)); + public static MTMVRelation generateMTMVRelation(Plan plan, ConnectContext connectContext) { + return new MTMVRelation(getBaseTables(plan, true, connectContext), + getBaseTables(plan, false, connectContext), getBaseViews(plan)); } - private static Set getBaseTables(Plan plan, boolean expand) { + private static Set getBaseTables(Plan plan, boolean expand, ConnectContext connectContext) { TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( com.google.common.collect.Sets - .newHashSet(TableType.values()), expand); + .newHashSet(TableType.values()), expand, connectContext); plan.accept(TableCollector.INSTANCE, collectorContext); Set collectedTables = collectorContext.getCollectedTables(); return transferTableIfToInfo(collectedTables); @@ -140,7 +133,7 @@ private static Set transferTableIfToInfo(Set tables) { return result; } - private static Plan getPlanBySql(String querySql, ConnectContext ctx) { + private static Plan getAnalyzePlanBySql(String querySql, ConnectContext ctx) { List statements; try { statements = new NereidsParser().parseSQL(querySql); @@ -153,7 +146,7 @@ private static Plan getPlanBySql(String querySql, ConnectContext ctx) { ctx.setStatementContext(new StatementContext()); try { NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - return planner.planWithLock(logicalPlan, PhysicalProperties.ANY, ExplainLevel.NONE); + return planner.planWithLock(logicalPlan, PhysicalProperties.ANY, ExplainLevel.ANALYZED_PLAN); } finally { ctx.setStatementContext(original); } 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 d365ff912de334..c9985911670878 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 @@ -719,11 +719,6 @@ public String getExplainString(ExplainOptions explainOptions) { return plan; } - @Override - public boolean isBlockQuery() { - return true; - } - @Override public DescriptorTable getDescTable() { return descTable; 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 ce897adf79c3b9..175b623467af28 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 @@ -174,6 +174,8 @@ public class StatementContext implements Closeable { private String disableJoinReorderReason; + private boolean privChecked; + private final Map snapshots = Maps.newHashMap(); public StatementContext() { @@ -599,4 +601,12 @@ public Optional getDisableJoinReorderReason() { public void setDisableJoinReorderReason(String disableJoinReorderReason) { this.disableJoinReorderReason = disableJoinReorderReason; } + + public boolean isPrivChecked() { + return privChecked; + } + + public void setPrivChecked(boolean privChecked) { + this.privChecked = privChecked; + } } 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 3a31381854c6c4..7f4d23c61303b3 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 @@ -582,7 +582,8 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla } else if (table instanceof TrinoConnectorExternalTable) { scanNode = new TrinoConnectorScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else if (table instanceof MaxComputeExternalTable) { - scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, + fileScan.getSelectedPartitions(), false); } else if (table instanceof LakeSoulExternalTable) { scanNode = new LakeSoulScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else { @@ -651,10 +652,12 @@ public PlanFragment visitPhysicalHudiScan(PhysicalHudiScan fileScan, PlanTransla + " for Hudi table"); PhysicalHudiScan hudiScan = (PhysicalHudiScan) fileScan; ScanNode scanNode = new HudiScanNode(context.nextPlanNodeId(), tupleDescriptor, false, - hudiScan.getScanParams(), hudiScan.getIncrementalRelation()); + hudiScan.getScanParams(), hudiScan.getIncrementalRelation(), ConnectContext.get().getSessionVariable()); if (fileScan.getTableSnapshot().isPresent()) { ((FileQueryScanNode) scanNode).setQueryTableSnapshot(fileScan.getTableSnapshot().get()); } + HudiScanNode hudiScanNode = (HudiScanNode) scanNode; + hudiScanNode.setSelectedPartitions(fileScan.getSelectedPartitions()); return getPlanFragmentForPhysicalFileScan(fileScan, context, scanNode, table, tupleDescriptor); } 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 4feeb6439e46c8..920f08cd9c3d04 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 @@ -64,6 +64,7 @@ import org.apache.doris.nereids.rules.rewrite.EliminateFilter; import org.apache.doris.nereids.rules.rewrite.EliminateGroupBy; import org.apache.doris.nereids.rules.rewrite.EliminateGroupByKey; +import org.apache.doris.nereids.rules.rewrite.EliminateGroupByKeyByUniform; import org.apache.doris.nereids.rules.rewrite.EliminateJoinByFK; import org.apache.doris.nereids.rules.rewrite.EliminateJoinByUnique; import org.apache.doris.nereids.rules.rewrite.EliminateJoinCondition; @@ -112,6 +113,7 @@ import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoinOnPkFk; import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoinOneSide; +import org.apache.doris.nereids.rules.rewrite.PushDownAggWithDistinctThroughJoinOneSide; import org.apache.doris.nereids.rules.rewrite.PushDownDistinctThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughProject; import org.apache.doris.nereids.rules.rewrite.PushDownLimit; @@ -343,6 +345,7 @@ public class Rewriter extends AbstractBatchJobExecutor { topic("Eager aggregation", costBased(topDown( + new PushDownAggWithDistinctThroughJoinOneSide(), new PushDownAggThroughJoinOneSide(), new PushDownAggThroughJoin() )), @@ -356,6 +359,7 @@ public class Rewriter extends AbstractBatchJobExecutor { topDown(new EliminateJoinByUnique()) ), topic("eliminate Aggregate according to fd items", + custom(RuleType.ELIMINATE_GROUP_BY_KEY_BY_UNIFORM, EliminateGroupByKeyByUniform::new), topDown(new EliminateGroupByKey()), topDown(new PushDownAggThroughJoinOnPkFk()), topDown(new PullUpJoinFromUnionAll()) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperElement.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperElement.java new file mode 100644 index 00000000000000..6d8d7c6326c0d9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperElement.java @@ -0,0 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.jobs.joinorder.hypergraph; + +/** + * This is the common base class for all + * */ +public interface HyperElement { + + // Get the references nodes + long getReferenceNodes(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java index 35694169498870..f75ed83250119a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.jobs.joinorder.hypergraph.edge; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; @@ -32,7 +33,7 @@ /** * Edge in HyperGraph */ -public abstract class Edge { +public abstract class Edge implements HyperElement { private final int index; private final double selectivity; @@ -42,7 +43,9 @@ public abstract class Edge { // added by the graph simplifier. private final long leftRequiredNodes; private final long rightRequiredNodes; + // The nodes needed which to prevent wrong association or l-association private long leftExtendedNodes; + // The nodes needed which to prevent wrong association or r-association private long rightExtendedNodes; // record the left child edges and right child edges in origin plan tree @@ -53,8 +56,11 @@ public abstract class Edge { private final BitSet curOperatorEdges = new BitSet(); // record all sub nodes behind in this operator. It's T function in paper private final long subTreeNodes; - + // The edges which prevents association or l-association when join edge + // and prevents push down or pull up when filter edge in the left of edge private final Set leftRejectEdges; + // The edges which prevents association or r-association + // and prevents push down or pull up when filter edge in the right of edge private final Set rightRejectEdges; /** @@ -187,6 +193,7 @@ public boolean isSub(Edge edge) { return LongBitmap.isSubset(getReferenceNodes(), otherBitmap); } + @Override public long getReferenceNodes() { return LongBitmap.newBitmapUnion(leftExtendedNodes, rightExtendedNodes); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java index a4a64e0449deee..686576de771d94 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.jobs.joinorder.hypergraph.node; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.Edge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.FilterEdge; @@ -33,7 +34,7 @@ /** * HyperGraph Node. */ -public class AbstractNode { +public class AbstractNode implements HyperElement { protected final int index; protected final List joinEdges; protected final List filterEdges; @@ -65,6 +66,11 @@ public List getEdges() { .build(); } + @Override + public long getReferenceNodes() { + return getNodeMap(); + } + public int getIndex() { return index; } 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 f29662a821ef8a..ccfbb428678999 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 @@ -3413,10 +3413,16 @@ private Expression withPredicate(Expression valueExpression, PredicateContext ct Expression outExpression; switch (ctx.kind.getType()) { case DorisParser.BETWEEN: - outExpression = new And( - new GreaterThanEqual(valueExpression, getExpression(ctx.lower)), - new LessThanEqual(valueExpression, getExpression(ctx.upper)) - ); + Expression lower = getExpression(ctx.lower); + Expression upper = getExpression(ctx.upper); + if (lower.equals(upper)) { + outExpression = new EqualTo(valueExpression, lower); + } else { + outExpression = new And( + new GreaterThanEqual(valueExpression, getExpression(ctx.lower)), + new LessThanEqual(valueExpression, getExpression(ctx.upper)) + ); + } break; case DorisParser.LIKE: outExpression = new Like( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java index e97fad6f479047..ff4756979e450e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DataTrait.java @@ -17,18 +17,23 @@ package org.apache.doris.nereids.properties; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.util.ImmutableEqualSet; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -46,16 +51,16 @@ public class DataTrait { public static final DataTrait EMPTY_TRAIT - = new DataTrait(new NestedSet().toImmutable(), - new NestedSet().toImmutable(), new ImmutableSet.Builder().build(), + = new DataTrait(new UniqueDescription().toImmutable(), + new UniformDescription().toImmutable(), new ImmutableSet.Builder().build(), ImmutableEqualSet.empty(), new FuncDepsDG.Builder().build()); - private final NestedSet uniqueSet; - private final NestedSet uniformSet; + private final UniqueDescription uniqueSet; + private final UniformDescription uniformSet; private final ImmutableSet fdItems; private final ImmutableEqualSet equalSet; private final FuncDepsDG fdDg; - private DataTrait(NestedSet uniqueSet, NestedSet uniformSet, ImmutableSet fdItems, + private DataTrait(UniqueDescription uniqueSet, UniformDescription uniformSet, ImmutableSet fdItems, ImmutableEqualSet equalSet, FuncDepsDG fdDg) { this.uniqueSet = uniqueSet; this.uniformSet = uniformSet; @@ -86,8 +91,7 @@ public boolean isUniform(Slot slot) { } public boolean isUniform(Set slotSet) { - return !slotSet.isEmpty() - && uniformSet.slots.containsAll(slotSet); + return uniformSet.contains(slotSet); } public boolean isUniqueAndNotNull(Slot slot) { @@ -102,11 +106,25 @@ public boolean isUniqueAndNotNull(Set slotSet) { } public boolean isUniformAndNotNull(Slot slot) { - return !slot.nullable() && isUniform(slot); + return uniformSet.isUniformAndNotNull(slot); } + /** isUniformAndNotNull for slot set */ public boolean isUniformAndNotNull(ImmutableSet slotSet) { - return slotSet.stream().noneMatch(Slot::nullable) && isUniform(slotSet); + for (Slot slot : slotSet) { + if (!uniformSet.isUniformAndNotNull(slot)) { + return false; + } + } + return true; + } + + public boolean isUniformAndHasConstValue(Slot slot) { + return uniformSet.isUniformAndHasConstValue(slot); + } + + public Optional getUniformValue(Slot slot) { + return uniformSet.slotUniformValue.get(slot); } public boolean isNullSafeEqual(Slot l, Slot r) { @@ -143,23 +161,23 @@ public String toString() { * Builder of trait */ public static class Builder { - private final NestedSet uniqueSet; - private final NestedSet uniformSet; + private final UniqueDescription uniqueSet; + private final UniformDescription uniformSet; private ImmutableSet fdItems; private final ImmutableEqualSet.Builder equalSetBuilder; private final FuncDepsDG.Builder fdDgBuilder; public Builder() { - uniqueSet = new NestedSet(); - uniformSet = new NestedSet(); + uniqueSet = new UniqueDescription(); + uniformSet = new UniformDescription(); fdItems = new ImmutableSet.Builder().build(); equalSetBuilder = new ImmutableEqualSet.Builder<>(); fdDgBuilder = new FuncDepsDG.Builder(); } public Builder(DataTrait other) { - this.uniformSet = new NestedSet(other.uniformSet); - this.uniqueSet = new NestedSet(other.uniqueSet); + this.uniformSet = new UniformDescription(other.uniformSet); + this.uniqueSet = new UniqueDescription(other.uniqueSet); this.fdItems = ImmutableSet.copyOf(other.fdItems); equalSetBuilder = new ImmutableEqualSet.Builder<>(other.equalSet); fdDgBuilder = new FuncDepsDG.Builder(other.fdDg); @@ -173,6 +191,14 @@ public void addUniformSlot(DataTrait dataTrait) { uniformSet.add(dataTrait.uniformSet); } + public void addUniformSlotForOuterJoinNullableSide(DataTrait dataTrait) { + uniformSet.addUniformSlotForOuterJoinNullableSide(dataTrait.uniformSet); + } + + public void addUniformSlotAndLiteral(Slot slot, Expression literal) { + uniformSet.add(slot, literal); + } + public void addUniqueSlot(Slot slot) { uniqueSet.add(slot); } @@ -261,8 +287,21 @@ public void addUniqueByEqualSet(Set equalSet) { * if there is a uniform slot in the equivalence set, then all slots of an equivalence set are uniform */ public void addUniformByEqualSet(Set equalSet) { - if (uniformSet.isIntersect(uniformSet.slots, equalSet)) { - uniformSet.slots.addAll(equalSet); + List intersectionList = uniformSet.slotUniformValue.keySet().stream() + .filter(equalSet::contains) + .collect(Collectors.toList()); + if (intersectionList.isEmpty()) { + return; + } + Expression expr = null; + for (Slot slot : intersectionList) { + if (uniformSet.slotUniformValue.get(slot).isPresent()) { + expr = uniformSet.slotUniformValue.get(slot).get(); + break; + } + } + for (Slot equal : equalSet) { + uniformSet.add(equal, expr); } } @@ -293,9 +332,11 @@ public List> getAllUniqueAndNotNull() { */ public List> getAllUniformAndNotNull() { List> res = new ArrayList<>(); - for (Slot s : uniformSet.slots) { - if (!s.nullable()) { - res.add(ImmutableSet.of(s)); + for (Map.Entry> entry : uniformSet.slotUniformValue.entrySet()) { + if (!entry.getKey().nullable()) { + res.add(ImmutableSet.of(entry.getKey())); + } else if (entry.getValue().isPresent() && !entry.getValue().get().nullable()) { + res.add(ImmutableSet.of(entry.getKey())); } } return res; @@ -338,21 +379,21 @@ public void replaceFuncDepsBy(Map replaceMap) { } } - static class NestedSet { + static class UniqueDescription { Set slots; Set> slotSets; - NestedSet() { + UniqueDescription() { slots = new HashSet<>(); slotSets = new HashSet<>(); } - NestedSet(NestedSet o) { + UniqueDescription(UniqueDescription o) { this.slots = new HashSet<>(o.slots); this.slotSets = new HashSet<>(o.slotSets); } - NestedSet(Set slots, Set> slotSets) { + UniqueDescription(Set slots, Set> slotSets) { this.slots = slots; this.slotSets = slotSets; } @@ -408,9 +449,9 @@ public void add(ImmutableSet slotSet) { slotSets.add(slotSet); } - public void add(NestedSet nestedSet) { - slots.addAll(nestedSet.slots); - slotSets.addAll(nestedSet.slotSets); + public void add(UniqueDescription uniqueDescription) { + slots.addAll(uniqueDescription.slots); + slotSets.addAll(uniqueDescription.slotSets); } public boolean isIntersect(Set set1, Set set2) { @@ -446,8 +487,120 @@ public void replace(Map replaceMap) { .collect(Collectors.toSet()); } - public NestedSet toImmutable() { - return new NestedSet(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets)); + public UniqueDescription toImmutable() { + return new UniqueDescription(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets)); + } + } + + static class UniformDescription { + // slot and its uniform expression(literal or const expression) + // some slot can get uniform values, others can not. + // e.g.select a from t where a=10 group by a, b; + // in LogicalAggregate, a UniformDescription with map {a : 10} can be obtained. + // which means a is uniform and the uniform value is 10. + Map> slotUniformValue; + + public UniformDescription() { + slotUniformValue = new LinkedHashMap<>(); + } + + public UniformDescription(UniformDescription ud) { + slotUniformValue = new LinkedHashMap<>(ud.slotUniformValue); + } + + public UniformDescription(Map> slotUniformValue) { + this.slotUniformValue = slotUniformValue; + } + + public UniformDescription toImmutable() { + return new UniformDescription(ImmutableMap.copyOf(slotUniformValue)); + } + + public boolean isEmpty() { + return slotUniformValue.isEmpty(); + } + + public boolean contains(Slot slot) { + return slotUniformValue.containsKey(slot); + } + + public boolean contains(Set slots) { + return !slots.isEmpty() && slotUniformValue.keySet().containsAll(slots); + } + + public void add(Slot slot) { + slotUniformValue.putIfAbsent(slot, Optional.empty()); + } + + public void add(Set slots) { + for (Slot s : slots) { + slotUniformValue.putIfAbsent(s, Optional.empty()); + } + } + + public void add(UniformDescription ud) { + slotUniformValue.putAll(ud.slotUniformValue); + for (Map.Entry> entry : ud.slotUniformValue.entrySet()) { + add(entry.getKey(), entry.getValue().orElse(null)); + } + } + + public void add(Slot slot, Expression literal) { + if (null == literal) { + slotUniformValue.putIfAbsent(slot, Optional.empty()); + } else { + slotUniformValue.put(slot, Optional.of(literal)); + } + } + + public void addUniformSlotForOuterJoinNullableSide(UniformDescription ud) { + for (Map.Entry> entry : ud.slotUniformValue.entrySet()) { + if ((!entry.getValue().isPresent() && entry.getKey().nullable()) + || (entry.getValue().isPresent() && entry.getValue().get() instanceof NullLiteral)) { + add(entry.getKey(), entry.getValue().orElse(null)); + } + } + } + + public void removeNotContain(Set slotSet) { + if (slotSet.isEmpty()) { + return; + } + Map> newSlotUniformValue = new LinkedHashMap<>(); + for (Map.Entry> entry : slotUniformValue.entrySet()) { + if (slotSet.contains(entry.getKey())) { + newSlotUniformValue.put(entry.getKey(), entry.getValue()); + } + } + this.slotUniformValue = newSlotUniformValue; + } + + public void replace(Map replaceMap) { + Map> newSlotUniformValue = new LinkedHashMap<>(); + for (Map.Entry> entry : slotUniformValue.entrySet()) { + Slot newKey = replaceMap.getOrDefault(entry.getKey(), entry.getKey()); + newSlotUniformValue.put(newKey, entry.getValue()); + } + slotUniformValue = newSlotUniformValue; + } + + // The current implementation logic is: if a slot key exists in map slotUniformValue, + // its value is present and is not nullable, + // or if a slot key exists in map slotUniformValue and the slot is not nullable + // it indicates that this slot is uniform and not null. + public boolean isUniformAndNotNull(Slot slot) { + return slotUniformValue.containsKey(slot) + && (!slot.nullable() || slotUniformValue.get(slot).isPresent() + && !slotUniformValue.get(slot).get().nullable()); + } + + public boolean isUniformAndHasConstValue(Slot slot) { + return slotUniformValue.containsKey(slot) && slotUniformValue.get(slot).isPresent(); + } + + @Override + public String toString() { + return "{" + slotUniformValue + "}"; } } } 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 430d2b35b35727..e9e7810487e876 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 @@ -191,6 +191,7 @@ public enum RuleType { ELIMINATE_SORT(RuleTypeClass.REWRITE), PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE(RuleTypeClass.REWRITE), + PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE(RuleTypeClass.REWRITE), PUSH_DOWN_AGG_THROUGH_JOIN(RuleTypeClass.REWRITE), PUSH_DOWN_AGG_THROUGH_JOIN_ON_PKFK(RuleTypeClass.REWRITE), TRANSPOSE_LOGICAL_SEMI_JOIN_LOGICAL_JOIN(RuleTypeClass.REWRITE), @@ -209,6 +210,11 @@ public enum RuleType { REWRITE_SORT_EXPRESSION(RuleTypeClass.REWRITE), REWRITE_HAVING_EXPRESSION(RuleTypeClass.REWRITE), REWRITE_REPEAT_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_SINK_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_WINDOW_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_SET_OPERATION_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_PARTITION_TOPN_EXPRESSION(RuleTypeClass.REWRITE), + REWRITE_TOPN_EXPRESSION(RuleTypeClass.REWRITE), EXTRACT_FILTER_FROM_JOIN(RuleTypeClass.REWRITE), REORDER_JOIN(RuleTypeClass.REWRITE), MERGE_PERCENTILE_TO_ARRAY(RuleTypeClass.REWRITE), @@ -238,6 +244,7 @@ public enum RuleType { ELIMINATE_JOIN_BY_UK(RuleTypeClass.REWRITE), ELIMINATE_JOIN_BY_FK(RuleTypeClass.REWRITE), ELIMINATE_GROUP_BY_KEY(RuleTypeClass.REWRITE), + ELIMINATE_GROUP_BY_KEY_BY_UNIFORM(RuleTypeClass.REWRITE), ELIMINATE_FILTER_GROUP_BY_KEY(RuleTypeClass.REWRITE), ELIMINATE_DEDUP_JOIN_CONDITION(RuleTypeClass.REWRITE), ELIMINATE_NULL_AWARE_LEFT_ANTI_JOIN(RuleTypeClass.REWRITE), 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 2bc9fb4c729d08..c803f0aac319a0 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 @@ -243,6 +243,10 @@ private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, unboundRelation.getTableSample()); } } + if (!tabletIds.isEmpty()) { + // This tabletIds is set manually, so need to set specifiedTabletIds + scan = scan.withManuallySpecifiedTabletIds(tabletIds); + } if (needGenerateLogicalAggForRandomDistAggTable(scan)) { // it's a random distribution agg table // add agg on olap scan 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 17e7d098cad552..d64c7f2baed6a4 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 @@ -42,9 +42,9 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; -import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.NotNullableAggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.AssertTrue; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; @@ -424,9 +424,9 @@ private Pair> addApply(SubqueryExpr subquery, Map replaceMap = new HashMap<>(); NamedExpression agg = ((ScalarSubquery) subquery).getTopLevelScalarAggFunction().get(); if (agg instanceof Alias) { - if (((Alias) agg).child() instanceof AlwaysNotNullable) { - AlwaysNotNullable notNullableAggFunc = - (AlwaysNotNullable) ((Alias) agg).child(); + if (((Alias) agg).child() instanceof NotNullableAggregateFunction) { + NotNullableAggregateFunction notNullableAggFunc = + (NotNullableAggregateFunction) ((Alias) agg).child(); if (subquery.getQueryPlan() instanceof LogicalProject) { LogicalProject logicalProject = (LogicalProject) subquery.getQueryPlan(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java index 743003dca6bb81..5a854f085b2f77 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -487,27 +487,37 @@ private static boolean isGroupByEqualsByFunctionDependency( if (!viewShuttledExprQueryBasedSet.containsAll(queryGroupShuttledExpression)) { return false; } - Set viewShouldUniformExpressionSet = new HashSet<>(); + Set viewShouldRemovedExpressionSet = new HashSet<>(); + Set viewScanShouldReservedExpressionSet = new HashSet<>(); // calc the group by expr which is needed to roll up and should be uniform - for (Map.Entry expressionEntry : + for (Map.Entry expressionMappingEntry : viewShuttledExprQueryBasedToViewGroupByExprMap.entrySet()) { - if (queryGroupShuttledExpression.contains(expressionEntry.getKey())) { - // the group expr which query has, do not require uniform - continue; + if (queryGroupShuttledExpression.contains(expressionMappingEntry.getKey())) { + // the group expr which query has, do not require eliminate + viewScanShouldReservedExpressionSet.add( + viewShuttledExprToScanExprMapping.get(expressionMappingEntry.getValue())); + } else { + // the view expression which is more than query's expression, should try to eliminate + viewShouldRemovedExpressionSet.add(expressionMappingEntry.getValue()); } - viewShouldUniformExpressionSet.add(expressionEntry.getValue()); } DataTrait dataTrait = tempRewrittenPlan.computeDataTrait(); - for (Expression shouldUniformExpr : viewShouldUniformExpressionSet) { - Expression viewScanExpression = viewShuttledExprToScanExprMapping.get(shouldUniformExpr); + for (Expression viewShouldRemovedExpr : viewShouldRemovedExpressionSet) { + Expression viewScanExpression = viewShuttledExprToScanExprMapping.get(viewShouldRemovedExpr); if (viewScanExpression == null) { return false; } if (!(viewScanExpression instanceof Slot)) { return false; } - if (!dataTrait.isUniform((Slot) viewScanExpression)) { + if (!dataTrait.isUniform((Slot) viewScanExpression) + && Sets.intersection(dataTrait.calEqualSet((Slot) viewScanExpression), + viewScanShouldReservedExpressionSet).isEmpty()) { + // Such as query is l_orderkey#0, l_linenumber#1, o_custkey#17, l_partkey#2 + // view is ps_partkey#25, o_orderkey#16, l_orderkey#0, l_linenumber#1, o_custkey#17, l_partkey#2 + // If want to check the group by equality, ps_partkey#25, o_orderkey#16 should be uniform + // or should be equal any of [ l_orderkey#0, l_linenumber#1, o_custkey#17, l_partkey#2] return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java index 868f97949c0705..22282a2351627b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java @@ -19,12 +19,14 @@ import org.apache.doris.common.Pair; import org.apache.doris.nereids.jobs.joinorder.hypergraph.ConflictRulesMaker; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperGraph; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.Edge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.FilterEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.JoinEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.ExpressionPosition; import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughJoin; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -35,18 +37,23 @@ import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.JoinUtils; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -75,9 +82,9 @@ public class HyperGraphComparator { private final Map> pullUpViewExprWithEdge = new HashMap<>(); private final LogicalCompatibilityContext logicalCompatibilityContext; // this records the slots which needs to reject null - // the key is the target join which should reject null, the value is a pair, the first value of the pair is the - // join type, the second value is also a pair which left represents the slots in the left of join that should - // reject null, right represents the slots in the right of join that should reject null. + // the key is the view join edge which should reject null, the value is a pair, the first value of the pair is the + // query join type, the second value is also a pair which left represents the slots in the left of view join that + // should reject null, right represents the slots in the right of view join that should reject null. private final Map, Set>>> inferredViewEdgeWithCond = new HashMap<>(); private List viewJoinEdgesAfterInferring; private List viewFilterEdgesAfterInferring; @@ -245,9 +252,17 @@ private boolean compareNodeWithExpr(StructInfoNode query, StructInfoNode view) { } int size = queryExprSetList.size(); for (int i = 0; i < size; i++) { - Set mappingQueryExprSet = queryExprSetList.get(i).stream() - .map(logicalCompatibilityContext::getViewNodeExprFromQuery) - .collect(Collectors.toSet()); + Set queryExpressions = queryExprSetList.get(i); + Set mappingQueryExprSet = new HashSet<>(); + for (Expression queryExpression : queryExpressions) { + Optional mappingViewExprByQueryExpr = getMappingViewExprByQueryExpr(queryExpression, query, + this.logicalCompatibilityContext, + ExpressionPosition.NODE); + if (!mappingViewExprByQueryExpr.isPresent()) { + return false; + } + mappingQueryExprSet.add(mappingViewExprByQueryExpr.get()); + } if (!mappingQueryExprSet.equals(viewExprSetList.get(i))) { return false; } @@ -403,7 +418,10 @@ private Map constructQueryToViewJoinMapWithExpr() { if (edgeMap.containsKey(entry.getValue())) { continue; } - Expression viewExpr = logicalCompatibilityContext.getViewJoinExprFromQuery(entry.getKey()); + Expression viewExpr = getMappingViewExprByQueryExpr(entry.getKey(), + entry.getValue(), + logicalCompatibilityContext, + ExpressionPosition.JOIN_EDGE).orElse(null); if (viewExprToEdge.containsKey(viewExpr)) { edgeMap.put(entry.getValue(), Objects.requireNonNull(viewExprToEdge.get(viewExpr))); } @@ -411,25 +429,76 @@ private Map constructQueryToViewJoinMapWithExpr() { return edgeMap; } + // Such as the filter as following, their expression is same, but should be different filter edge + // Only construct edge that can mapping, the edges which can not mapping would be handled by buildComparisonRes + // LogicalJoin[569] + // |--LogicalProject[567] + // | +--LogicalFilter[566] ( predicates=(l_orderkey#10 IS NULL OR ( not (l_orderkey#10 = 1))) ) + // | +--LogicalJoin[565] + // | |--LogicalProject[562] + // | | +--LogicalOlapScan + // | +--LogicalProject[564] + // | +--LogicalFilter[563] ( predicates=(l_orderkey#10 IS NULL OR ( not (l_orderkey#10 = 1)))) + // | +--LogicalOlapScan + // +--LogicalProject[568] + // +--LogicalOlapScan private Map constructQueryToViewFilterMapWithExpr() { - Map viewExprToEdge = getViewFilterEdges().stream() - .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) - .collect(ImmutableMap.toImmutableMap(p -> p.first, p -> p.second)); - Map queryExprToEdge = getQueryFilterEdges().stream() + Multimap viewExprToEdge = HashMultimap.create(); + getViewFilterEdges().stream() .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) - .collect(ImmutableMap.toImmutableMap(p -> p.first, p -> p.second)); + .forEach(pair -> viewExprToEdge.put(pair.key(), pair.value())); - HashMap edgeMap = new HashMap<>(); - for (Entry entry : queryExprToEdge.entrySet()) { - if (edgeMap.containsKey(entry.getValue())) { - continue; + Multimap queryExprToEdge = HashMultimap.create(); + getQueryFilterEdges().stream() + .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) + .forEach(pair -> queryExprToEdge.put(pair.key(), pair.value())); + + HashMap queryToViewEdgeMap = new HashMap<>(); + for (Entry> entry : queryExprToEdge.asMap().entrySet()) { + Expression queryExprViewBased = null; + for (Edge queryEdge : entry.getValue()) { + queryExprViewBased = getMappingViewExprByQueryExpr(entry.getKey(), + queryEdge, + logicalCompatibilityContext, + ExpressionPosition.FILTER_EDGE).orElse(null); + if (queryExprViewBased == null) { + continue; + } + Collection viewEdges = viewExprToEdge.get(queryExprViewBased); + if (viewEdges.isEmpty()) { + continue; + } + for (Edge viewEdge : viewEdges) { + if (!isSubTreeNodesEquals(queryEdge, viewEdge, logicalCompatibilityContext)) { + // Such as query filter edge is <{1} --FILTER-- {}> but view filter edge is + // <{0, 1} --FILTER-- {}>, though they are all + // l_orderkey#10 IS NULL OR ( not (l_orderkey#10 = 1)) but they are different actually + continue; + } + queryToViewEdgeMap.put(queryEdge, viewEdge); + } } - Expression viewExpr = logicalCompatibilityContext.getViewFilterExprFromQuery(entry.getKey()); - if (viewExprToEdge.containsKey(viewExpr)) { - edgeMap.put(entry.getValue(), Objects.requireNonNull(viewExprToEdge.get(viewExpr))); + } + return queryToViewEdgeMap; + } + + private static boolean isSubTreeNodesEquals(Edge queryEdge, Edge viewEdge, + LogicalCompatibilityContext logicalCompatibilityContext) { + if (!(queryEdge instanceof FilterEdge) || !(viewEdge instanceof FilterEdge)) { + return false; + } + // subTreeNodes should be equal + BiMap queryToViewNodeIdMapping = + logicalCompatibilityContext.getQueryToViewNodeIDMapping(); + List queryNodeIndexViewBasedList = new ArrayList<>(); + for (int queryNodeIndex : LongBitmap.getIterator(queryEdge.getSubTreeNodes())) { + Integer queryNodeIndexViewBased = queryToViewNodeIdMapping.get(queryNodeIndex); + if (queryNodeIndexViewBased == null) { + return false; } + queryNodeIndexViewBasedList.add(queryNodeIndexViewBased); } - return edgeMap; + return LongBitmap.newBitmap(queryNodeIndexViewBasedList) == viewEdge.getSubTreeNodes(); } private void refreshViewEdges() { @@ -463,17 +532,17 @@ private boolean compareEdgeWithNode(Edge query, Edge view) { } private boolean compareFilterEdgeWithNode(FilterEdge query, FilterEdge view) { - return rewriteQueryNodeMap(query.getReferenceNodes()) == view.getReferenceNodes(); + return getViewNodesByQuery(query.getReferenceNodes()) == view.getReferenceNodes(); } private boolean compareJoinEdgeWithNode(JoinEdge query, JoinEdge view) { boolean res = false; if (query.getJoinType().swap() == view.getJoinType()) { - res |= rewriteQueryNodeMap(query.getLeftExtendedNodes()) == view.getRightExtendedNodes() - && rewriteQueryNodeMap(query.getRightExtendedNodes()) == view.getLeftExtendedNodes(); + res |= getViewNodesByQuery(query.getLeftExtendedNodes()) == view.getRightExtendedNodes() + && getViewNodesByQuery(query.getRightExtendedNodes()) == view.getLeftExtendedNodes(); } - res |= rewriteQueryNodeMap(query.getLeftExtendedNodes()) == view.getLeftExtendedNodes() - && rewriteQueryNodeMap(query.getRightExtendedNodes()) == view.getRightExtendedNodes(); + res |= getViewNodesByQuery(query.getLeftExtendedNodes()) == view.getLeftExtendedNodes() + && getViewNodesByQuery(query.getRightExtendedNodes()) == view.getRightExtendedNodes(); return res; } @@ -496,8 +565,8 @@ private boolean compareJoinEdgeOrInfer(JoinEdge query, JoinEdge view) { } private boolean tryInferEdge(JoinEdge query, JoinEdge view) { - if (rewriteQueryNodeMap(query.getLeftRequiredNodes()) != view.getLeftRequiredNodes() - || rewriteQueryNodeMap(query.getRightRequiredNodes()) != view.getRightRequiredNodes()) { + if (getViewNodesByQuery(query.getLeftRequiredNodes()) != view.getLeftRequiredNodes() + || getViewNodesByQuery(query.getRightRequiredNodes()) != view.getRightRequiredNodes()) { return false; } if (!query.getJoinType().equals(view.getJoinType())) { @@ -518,7 +587,7 @@ private boolean tryInferEdge(JoinEdge query, JoinEdge view) { return true; } - private long rewriteQueryNodeMap(long bitmap) { + private long getViewNodesByQuery(long bitmap) { long newBitmap = LongBitmap.newBitmap(); for (int i : LongBitmap.getIterator(bitmap)) { int newIdx = getQueryToViewNodeIdMap().getOrDefault(i, 0); @@ -527,6 +596,35 @@ private long rewriteQueryNodeMap(long bitmap) { return newBitmap; } + private Optional getMappingViewExprByQueryExpr(Expression queryExpression, + HyperElement queryExpressionBelongedHyperElement, + LogicalCompatibilityContext context, + ExpressionPosition expressionPosition) { + Expression queryShuttledExpr; + Collection> viewExpressions; + if (ExpressionPosition.JOIN_EDGE.equals(expressionPosition)) { + queryShuttledExpr = context.getQueryJoinShuttledExpr(queryExpression); + viewExpressions = context.getViewJoinExprFromQuery(queryShuttledExpr); + } else if (ExpressionPosition.FILTER_EDGE.equals(expressionPosition)) { + queryShuttledExpr = context.getQueryFilterShuttledExpr(queryExpression); + viewExpressions = context.getViewFilterExprFromQuery(queryShuttledExpr); + } else { + queryShuttledExpr = context.getQueryNodeShuttledExpr(queryExpression); + viewExpressions = context.getViewNodeExprFromQuery(queryShuttledExpr); + } + if (viewExpressions.size() == 1) { + return Optional.of(viewExpressions.iterator().next().key()); + } + long queryReferenceNodes = queryExpressionBelongedHyperElement.getReferenceNodes(); + long viewReferenceNodes = getViewNodesByQuery(queryReferenceNodes); + for (Pair viewExpressionPair : viewExpressions) { + if (viewExpressionPair.value().getReferenceNodes() == viewReferenceNodes) { + return Optional.of(viewExpressionPair.key()); + } + } + return Optional.empty(); + } + private void compareJoinEdgeWithExpr(Edge query, Edge view) { Set queryExprSet = query.getExpressionSet(); Set viewExprSet = view.getExpressionSet(); @@ -534,7 +632,10 @@ private void compareJoinEdgeWithExpr(Edge query, Edge view) { Set exprMappedOfView = new HashSet<>(); List residualQueryExpr = new ArrayList<>(); for (Expression queryExpr : queryExprSet) { - Expression viewExpr = logicalCompatibilityContext.getViewJoinExprFromQuery(queryExpr); + Expression viewExpr = getMappingViewExprByQueryExpr(queryExpr, + query, + logicalCompatibilityContext, + ExpressionPosition.JOIN_EDGE).orElse(null); if (viewExprSet.contains(viewExpr)) { exprMappedOfView.add(viewExpr); } else { @@ -553,7 +654,10 @@ private void compareFilterEdgeWithExpr(Edge query, Edge view) { Set exprMappedOfView = new HashSet<>(); List residualQueryExpr = new ArrayList<>(); for (Expression queryExpr : queryExprSet) { - Expression viewExpr = logicalCompatibilityContext.getViewFilterExprFromQuery(queryExpr); + Expression viewExpr = getMappingViewExprByQueryExpr(queryExpr, + query, + logicalCompatibilityContext, + ExpressionPosition.FILTER_EDGE).orElse(null); if (viewExprSet.contains(viewExpr)) { exprMappedOfView.add(viewExpr); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 2e8baecf165639..4f8198e0b3c0bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -79,13 +79,18 @@ public void initMaterializationContext(CascadesContext cascadesContext) { * @param cascadesContext current cascadesContext in the planner */ protected void doInitMaterializationContext(CascadesContext cascadesContext) { + if (cascadesContext.getConnectContext().getSessionVariable().isInDebugMode()) { + LOG.info(String.format("MaterializationContext init return because is in debug mode, current queryId is %s", + cascadesContext.getConnectContext().getQueryIdentifier())); + return; + } // Only collect the table or mv which query use directly, to avoid useless mv partition in rewrite - TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false); + // Keep use one connection context when in query, if new connect context, + // the ConnectionContext.get() will change + TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false, + cascadesContext.getConnectContext()); try { Plan rewritePlan = cascadesContext.getRewritePlan(); - // Keep use one connection context when in query, if new connect context, - // the ConnectionContext.get() will change - collectorContext.setConnectContext(cascadesContext.getConnectContext()); rewritePlan.accept(TableCollector.INSTANCE, collectorContext); } catch (Exception e) { LOG.warn(String.format("MaterializationContext init table collect fail, current queryId is %s", diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java index ca13c9701dabc2..77ab37873d06b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java @@ -17,6 +17,8 @@ package org.apache.doris.nereids.rules.exploration.mv; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.rules.exploration.mv.StructInfo.ExpressionPosition; @@ -36,8 +38,10 @@ import com.google.common.base.Suppliers; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; -import java.util.HashMap; +import java.util.Collection; import java.util.Map; import java.util.function.Supplier; @@ -48,11 +52,15 @@ public class LogicalCompatibilityContext { private final BiMap queryToViewNodeMapping; private final BiMap queryToViewNodeIDMapping; private final ObjectId planNodeId; - private final Supplier> queryToViewJoinEdgeExpressionMappingSupplier; - private final Supplier> queryToViewNodeExpressionMappingSupplier; - private final Supplier> queryToViewFilterEdgeExpressionMappingSupplier; - @Deprecated - private BiMap queryToViewAllExpressionMapping; + private final Supplier>> + queryToViewJoinEdgeExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledJoinExpressionMappingSupplier; + private final Supplier>> + queryToViewNodeExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledNodeExpressionMappingSupplier; + private final Supplier>> + queryToViewFilterEdgeExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledFilterExpressionMappingSupplier; /** * LogicalCompatibilityContext @@ -66,16 +74,25 @@ private LogicalCompatibilityContext(BiMap queryT queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.JOIN_EDGE), viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.JOIN_EDGE))); + this.queryToQueryShuttledJoinExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.JOIN_EDGE)); + this.queryToViewNodeExpressionMappingSupplier = Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.NODE), viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.NODE))); + this.queryToQueryShuttledNodeExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.NODE)); + this.queryToViewFilterEdgeExpressionMappingSupplier = Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.FILTER_EDGE), viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.FILTER_EDGE))); + this.queryToQueryShuttledFilterExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.FILTER_EDGE)); + this.queryToViewNodeMapping = queryToViewNodeMapping; this.queryToViewNodeIDMapping = HashBiMap.create(); queryToViewNodeMapping.forEach((k, v) -> queryToViewNodeIDMapping.put(k.getIndex(), v.getIndex())); @@ -92,18 +109,30 @@ public BiMap getQueryToViewNodeIDMapping() { return queryToViewNodeIDMapping; } - public Expression getViewJoinExprFromQuery(Expression queryJoinExpr) { + public Collection> getViewJoinExprFromQuery(Expression queryJoinExpr) { return queryToViewJoinEdgeExpressionMappingSupplier.get().get(queryJoinExpr); } - public Expression getViewFilterExprFromQuery(Expression queryJoinExpr) { + public Expression getQueryJoinShuttledExpr(Expression queryJoinExpr) { + return queryToQueryShuttledJoinExpressionMappingSupplier.get().get(queryJoinExpr); + } + + public Collection> getViewFilterExprFromQuery(Expression queryJoinExpr) { return queryToViewFilterEdgeExpressionMappingSupplier.get().get(queryJoinExpr); } - public Expression getViewNodeExprFromQuery(Expression queryJoinExpr) { + public Expression getQueryFilterShuttledExpr(Expression queryFilterExpr) { + return queryToQueryShuttledFilterExpressionMappingSupplier.get().get(queryFilterExpr); + } + + public Collection> getViewNodeExprFromQuery(Expression queryJoinExpr) { return queryToViewNodeExpressionMappingSupplier.get().get(queryJoinExpr); } + public Expression getQueryNodeShuttledExpr(Expression queryNodeExpr) { + return queryToQueryShuttledNodeExpressionMappingSupplier.get().get(queryNodeExpr); + } + /** * Generate logical compatibility context, * this make expression mapping between query and view by relation and the slot in relation mapping @@ -134,24 +163,31 @@ public static LogicalCompatibilityContext from(RelationMapping relationMapping, viewStructInfo); } - private static BiMap generateExpressionMapping( + /** + * The result is multimap + * the key is shuttled query expr + * the value is original view expr collection + * */ + private static Multimap> generateExpressionMapping( Map viewToQuerySlotMapping, - Map queryShuttledExprToExprMap, - Map viewShuttledExprToExprMap) { - final Map viewEdgeToConjunctsMapQueryBased = new HashMap<>(); - BiMap queryToViewEdgeMapping = HashBiMap.create(); + Multimap> queryShuttledExprToExprMap, + Multimap> viewShuttledExprToExprMap) { + Multimap> queryToViewEdgeMapping = HashMultimap.create(); if (queryShuttledExprToExprMap == null || viewShuttledExprToExprMap == null || queryShuttledExprToExprMap.isEmpty() || viewShuttledExprToExprMap.isEmpty()) { return queryToViewEdgeMapping; } + final Multimap> viewShuttledExprToExprMapQueryBased = + HashMultimap.create(); viewShuttledExprToExprMap.forEach((shuttledExpr, expr) -> { - viewEdgeToConjunctsMapQueryBased.put( + viewShuttledExprToExprMapQueryBased.put( orderSlotAsc(ExpressionUtils.replace(shuttledExpr, viewToQuerySlotMapping)), expr); }); - queryShuttledExprToExprMap.forEach((exprSet, edge) -> { - Expression viewExpr = viewEdgeToConjunctsMapQueryBased.get(orderSlotAsc(exprSet)); - if (viewExpr != null) { - queryToViewEdgeMapping.put(edge, viewExpr); + queryShuttledExprToExprMap.forEach((shuttledExpr, expr) -> { + Collection> viewExpressions = viewShuttledExprToExprMapQueryBased.get( + orderSlotAsc(shuttledExpr)); + if (viewExpressions != null) { + queryToViewEdgeMapping.putAll(shuttledExpr, viewExpressions); } }); return queryToViewEdgeMapping; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index 484abd11f01e72..4c5703e27687e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -343,11 +343,19 @@ public Plan visitLogicalResultSink(LogicalResultSink logicalResu ImmutableList.of(Rewriter.custom(RuleType.ELIMINATE_SORT, EliminateSort::new))).execute(); return childContext.getRewritePlan(); }, mvPlan, originPlan); - return new MTMVCache(mvPlan, originPlan, + return new MTMVCache(mvPlan, originPlan, planner.getAnalyzedPlan(), planner.getCascadesContext().getMemo().getRoot().getStatistics(), null); } - private static final class TableQueryOperatorChecker extends DefaultPlanVisitor { + /** + * Check the query if Contains query operator + * Such sql as following should return true + * select * from orders TABLET(10098) because TABLET(10098) should return true + * select * from orders_partition PARTITION (day_2) because PARTITION (day_2) + * select * from orders index query_index_test because index query_index_test + * select * from orders TABLESAMPLE(20 percent) because TABLESAMPLE(20 percent) + * */ + public static final class TableQueryOperatorChecker extends DefaultPlanVisitor { public static final TableQueryOperatorChecker INSTANCE = new TableQueryOperatorChecker(); @Override @@ -358,12 +366,20 @@ public Boolean visitLogicalRelation(LogicalRelation relation, Void context) { if (relation instanceof LogicalOlapScan) { LogicalOlapScan logicalOlapScan = (LogicalOlapScan) relation; if (logicalOlapScan.getTableSample().isPresent()) { + // Contain sample, select * from orders TABLESAMPLE(20 percent) return true; } - if (!logicalOlapScan.getSelectedTabletIds().isEmpty()) { + if (!logicalOlapScan.getManuallySpecifiedTabletIds().isEmpty()) { + // Contain tablets, select * from orders TABLET(10098) because TABLET(10098) return true; } if (!logicalOlapScan.getManuallySpecifiedPartitions().isEmpty()) { + // Contain specified partitions, select * from orders_partition PARTITION (day_2) + return true; + } + if (logicalOlapScan.getSelectedIndexId() != logicalOlapScan.getTable().getBaseIndexId()) { + // Contains select index or use sync mv in rbo rewrite + // select * from orders index query_index_test return true; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index 526ec7030d2db5..2e2119efe7176c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -23,11 +23,12 @@ import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.executor.Rewriter; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperGraph; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.JoinEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; -import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.TableQueryOperatorChecker; import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; import org.apache.doris.nereids.trees.copier.DeepCopierContext; import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; @@ -36,6 +37,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.plans.AbstractPlan; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.ObjectId; @@ -61,12 +63,15 @@ import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer; import org.apache.doris.nereids.util.ExpressionUtils; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; @@ -109,9 +114,23 @@ public class StructInfo { // split predicates is shuttled private SplitPredicate splitPredicate; private EquivalenceClass equivalenceClass; - // Key is the expression shuttled and the value is the origin expression + // For value of Map, the key is the position of expression + // the value is the expressions and the hyper element of expression pair + // Key of pair is the expression shuttled and the value is the origin expression and the hyper element it belonged + // Sometimes origin expressions are different and shuttled expression is same + // Such as origin expressions are l_partkey#0 > 1 and l_partkey#10 > 1 and shuttled expression is l_partkey#10 > 1 // this is for building LogicalCompatibilityContext later. - private final Map> shuttledExpressionsToExpressionsMap; + private final Map>> + shuttledExpressionsToExpressionsMap; + // For value of Map, the key is the position of expression + // the value is the original expression and shuttled expression map + // Such as origin expressions are l_partkey#0 > 1 and shuttled expression is l_partkey#10 > 1 + // the map would be {ExpressionPosition.FILTER, { + // l_partkey#0 > 1 : l_partkey#10 > 1 + // }} + // this is for building LogicalCompatibilityContext later. + private final Map> expressionToShuttledExpressionToMap; + // Record the exprId and the corresponding expr map, this is used by expression shuttled private final Map namedExprIdAndExprMapping; private final List planOutputShuttledExpressions; @@ -123,7 +142,9 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG Plan bottomPlan, List relations, Map relationIdStructInfoNodeMap, @Nullable Predicates predicates, - Map> shuttledExpressionsToExpressionsMap, + Map>> + shuttledExpressionsToExpressionsMap, + Map> expressionToShuttledExpressionToMap, Map namedExprIdAndExprMapping, BitSet tableIdSet, SplitPredicate splitPredicate, @@ -142,6 +163,7 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG this.splitPredicate = splitPredicate; this.equivalenceClass = equivalenceClass; this.shuttledExpressionsToExpressionsMap = shuttledExpressionsToExpressionsMap; + this.expressionToShuttledExpressionToMap = expressionToShuttledExpressionToMap; this.namedExprIdAndExprMapping = namedExprIdAndExprMapping; this.planOutputShuttledExpressions = planOutputShuttledExpressions; } @@ -152,7 +174,8 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG public StructInfo withPredicates(Predicates predicates) { return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, predicates, - this.shuttledExpressionsToExpressionsMap, this.namedExprIdAndExprMapping, this.tableBitSet, + this.shuttledExpressionsToExpressionsMap, this.expressionToShuttledExpressionToMap, + this.namedExprIdAndExprMapping, this.tableBitSet, null, null, this.planOutputShuttledExpressions); } @@ -162,13 +185,16 @@ public StructInfo withPredicates(Predicates predicates) { public StructInfo withTableBitSet(BitSet tableBitSet) { return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, this.predicates, - this.shuttledExpressionsToExpressionsMap, this.namedExprIdAndExprMapping, tableBitSet, + this.shuttledExpressionsToExpressionsMap, this.expressionToShuttledExpressionToMap, + this.namedExprIdAndExprMapping, tableBitSet, this.splitPredicate, this.equivalenceClass, this.planOutputShuttledExpressions); } private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, Plan topPlan, - Map> shuttledExpressionsToExpressionsMap, + Map>> + shuttledExpressionsToExpressionsMap, + Map> expressionToShuttledExpressionToMap, Map namedExprIdAndExprMapping, List relations, Map relationIdStructInfoNodeMap, @@ -196,8 +222,9 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, structInfoNode.getPlan().accept(ExpressionLineageReplacer.INSTANCE, replaceContext); // Replace expressions by expression map List replacedExpressions = replaceContext.getReplacedExpressions(); - putShuttledExpressionsToExpressionsMap(shuttledExpressionsToExpressionsMap, - ExpressionPosition.NODE, replacedExpressions.get(0), expression); + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, + ExpressionPosition.NODE, replacedExpressions.get(0), expression, node); // Record this, will be used in top level expression shuttle later, see the method // ExpressionLineageReplacer#visitGroupPlan namedExprIdAndExprMapping.putAll(replaceContext.getExprIdExpressionMap()); @@ -223,8 +250,10 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, // Replace expressions by expression map List replacedExpressions = replaceContext.getReplacedExpressions(); for (int i = 0; i < replacedExpressions.size(); i++) { - putShuttledExpressionsToExpressionsMap(shuttledExpressionsToExpressionsMap, - ExpressionPosition.JOIN_EDGE, replacedExpressions.get(i), joinConjunctExpressions.get(i)); + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, + ExpressionPosition.JOIN_EDGE, replacedExpressions.get(i), joinConjunctExpressions.get(i), + edge); } // Record this, will be used in top level expression shuttle later, see the method // ExpressionLineageReplacer#visitGroupPlan @@ -236,10 +265,11 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, filterExpressions.forEach(predicate -> { // this is used for LogicalCompatibilityContext ExpressionUtils.extractConjunction(predicate).forEach(expr -> - putShuttledExpressionsToExpressionsMap(shuttledExpressionsToExpressionsMap, + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, ExpressionPosition.FILTER_EDGE, ExpressionUtils.shuttleExpressionWithLineage(predicate, topPlan, new BitSet()), - predicate)); + predicate, filterEdge)); }); }); return true; @@ -311,11 +341,13 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable // collect struct info fromGraph List relationList = new ArrayList<>(); Map relationIdStructInfoNodeMap = new LinkedHashMap<>(); - Map> shuttledHashConjunctsToConjunctsMap = - new LinkedHashMap<>(); + Map>> + shuttledHashConjunctsToConjunctsMap = new LinkedHashMap<>(); Map namedExprIdAndExprMapping = new LinkedHashMap<>(); BitSet tableBitSet = new BitSet(); + Map> expressionToShuttledExpressionToMap = new HashMap<>(); boolean valid = collectStructInfoFromGraph(hyperGraph, topPlan, shuttledHashConjunctsToConjunctsMap, + expressionToShuttledExpressionToMap, namedExprIdAndExprMapping, relationList, relationIdStructInfoNodeMap, @@ -323,6 +355,11 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable cascadesContext); valid = valid && hyperGraph.getNodes().stream().allMatch(n -> ((StructInfoNode) n).getExpressions() != null); + // if relationList has any relation which contains table operator, + // such as query with sample, index, table, is invalid + boolean invalid = relationList.stream().anyMatch(relation -> + ((AbstractPlan) relation).accept(TableQueryOperatorChecker.INSTANCE, null)); + valid = valid && !invalid; // collect predicate from top plan which not in hyper graph Set topPlanPredicates = new LinkedHashSet<>(); topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates); @@ -332,19 +369,11 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable ExpressionUtils.shuttleExpressionWithLineage(originalPlan.getOutput(), originalPlan, new BitSet()); return new StructInfo(originalPlan, originalPlanId, hyperGraph, valid, topPlan, bottomPlan, relationList, relationIdStructInfoNodeMap, predicates, shuttledHashConjunctsToConjunctsMap, + expressionToShuttledExpressionToMap, namedExprIdAndExprMapping, tableBitSet, null, null, planOutputShuttledExpressions); } - /** - * Build Struct info from group. - * Maybe return multi structInfo when original plan already be rewritten by mv - */ - public static StructInfo of(Group group) { - // TODO build graph from original plan and get relations and predicates from graph - return null; - } - public List getRelations() { return relations; } @@ -401,21 +430,36 @@ public Map getRelationIdStructInfoNodeMap() { return relationIdStructInfoNodeMap; } - public Map> getShuttledExpressionsToExpressionsMap() { + public Map>> + getShuttledExpressionsToExpressionsMap() { return shuttledExpressionsToExpressionsMap; } - private static void putShuttledExpressionsToExpressionsMap( - Map> shuttledExpressionsToExpressionsMap, + public Map> getExpressionToShuttledExpressionToMap() { + return expressionToShuttledExpressionToMap; + } + + private static void putShuttledExpressionToExpressionsMap( + Map>> + shuttledExpressionsToExpressionsMap, + Map> expressionPositionToExpressionToMap, ExpressionPosition expressionPosition, - Expression key, Expression value) { - Map expressionExpressionMap = shuttledExpressionsToExpressionsMap.get( - expressionPosition); - if (expressionExpressionMap == null) { - expressionExpressionMap = new LinkedHashMap<>(); - shuttledExpressionsToExpressionsMap.put(expressionPosition, expressionExpressionMap); - } - expressionExpressionMap.put(key, value); + Expression shuttledExpression, Expression originalExpression, HyperElement valueBelongedElement) { + Multimap> shuttledExpressionToExpressionMap = + shuttledExpressionsToExpressionsMap.get(expressionPosition); + if (shuttledExpressionToExpressionMap == null) { + shuttledExpressionToExpressionMap = HashMultimap.create(); + shuttledExpressionsToExpressionsMap.put(expressionPosition, shuttledExpressionToExpressionMap); + } + shuttledExpressionToExpressionMap.put(shuttledExpression, Pair.of(originalExpression, valueBelongedElement)); + + Map originalExprToShuttledExprMap = + expressionPositionToExpressionToMap.get(expressionPosition); + if (originalExprToShuttledExprMap == null) { + originalExprToShuttledExprMap = new HashMap<>(); + expressionPositionToExpressionToMap.put(expressionPosition, originalExprToShuttledExprMap); + } + originalExprToShuttledExprMap.put(originalExpression, shuttledExpression); } public List getExpressions() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/mapping/SlotMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/mapping/SlotMapping.java index 882dec19ea82dd..6e481a101420a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/mapping/SlotMapping.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/mapping/SlotMapping.java @@ -19,7 +19,6 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.types.VariantType; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; @@ -78,19 +77,19 @@ public static SlotMapping generate(RelationMapping relationMapping) { for (List sourceSlotName : sourceSlotNameToSlotMap.keySet()) { Slot sourceSlot = sourceSlotNameToSlotMap.get(sourceSlotName); Slot targetSlot = targetSlotNameSlotMap.get(sourceSlotName); - // source slot can not map from target, bail out - if (targetSlot == null && !(((SlotReference) sourceSlot).getDataType() instanceof VariantType)) { - LOG.warn(String.format("SlotMapping generate is null, source relation is %s, " - + "target relation is %s", sourceRelation, targetRelation)); - return null; - } if (targetSlot == null) { + // there are two scenes in which targetSlot maybe null + // 1 // if variant, though can not map slot from query to view, but we maybe derive slot from query // variant self, such as query slot to view slot mapping is payload#4 -> payload#10 // and query has a variant which is payload['issue']['number']#20, this can not get from view. // in this scene, we can derive // payload['issue']['number']#20 -> element_at(element_at(payload#10, 'issue'), 'number') mapping // in expression rewrite + // 2 + // Maybe table add column after last refresh + LOG.warn(String.format("SlotMapping generate is null, source relation is %s, " + + "target relation is %s", sourceRelation, targetRelation)); continue; } relationSlotMap.put(MappedSlot.of(sourceSlot, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java index abf57057601dc8..81104f981022c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.rules.expression; import org.apache.doris.nereids.rules.expression.rules.ArrayContainToArrayOverlap; +import org.apache.doris.nereids.rules.expression.rules.BetweenToEqual; import org.apache.doris.nereids.rules.expression.rules.CaseWhenToIf; import org.apache.doris.nereids.rules.expression.rules.DateFunctionRewrite; import org.apache.doris.nereids.rules.expression.rules.DistinctPredicatesRule; @@ -53,7 +54,8 @@ public class ExpressionOptimization extends ExpressionRewrite { CaseWhenToIf.INSTANCE, TopnToMax.INSTANCE, NullSafeEqualToEqual.INSTANCE, - LikeToEqualRewrite.INSTANCE + LikeToEqualRewrite.INSTANCE, + BetweenToEqual.INSTANCE ) ); private static final ExpressionRuleExecutor EXECUTOR = new ExpressionRuleExecutor(OPTIMIZE_REWRITE_RULES); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java index 0eb4d3a79a0c96..277f1e6f2d8adc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java @@ -53,7 +53,7 @@ * expression of plan rewrite rule. */ public class ExpressionRewrite implements RewriteRuleFactory { - private final ExpressionRuleExecutor rewriter; + protected final ExpressionRuleExecutor rewriter; public ExpressionRewrite(ExpressionRewriteRule... rules) { this.rewriter = new ExpressionRuleExecutor(ImmutableList.copyOf(rules)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/BetweenToEqual.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/BetweenToEqual.java new file mode 100644 index 00000000000000..75c00541c63194 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/BetweenToEqual.java @@ -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. + +package org.apache.doris.nereids.rules.expression.rules; + +import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher; +import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; +import org.apache.doris.nereids.trees.expressions.LessThanEqual; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * f(A, B) between 1 and 1 => f(A, B) = 1 + * + */ +public class BetweenToEqual implements ExpressionPatternRuleFactory { + + public static BetweenToEqual INSTANCE = new BetweenToEqual(); + + @Override + public List> buildRules() { + return ImmutableList.of( + matchesType(And.class).then(BetweenToEqual::rewriteBetweenToEqual) + ); + } + + private static Expression rewriteBetweenToEqual(And and) { + List conjuncts = ExpressionUtils.extractConjunction(and); + Map> betweenCandidate = Maps.newHashMap(); + for (Expression conj : conjuncts) { + if (isCandidate(conj)) { + conj = normalizeCandidate((ComparisonPredicate) conj); + Expression varPart = conj.child(0); + betweenCandidate.computeIfAbsent(varPart, k -> Lists.newArrayList()); + betweenCandidate.get(varPart).add((ComparisonPredicate) conj); + } + } + List equals = Lists.newArrayList(); + List equalsKey = Lists.newArrayList(); + for (Expression varPart : betweenCandidate.keySet()) { + List candidates = betweenCandidate.get(varPart); + if (candidates.size() == 2 && greaterEqualAndLessEqual(candidates.get(0), candidates.get(1))) { + if (candidates.get(0).child(1).equals(candidates.get(1).child(1))) { + equals.add(new EqualTo(candidates.get(0).child(0), candidates.get(0).child(1))); + equalsKey.add(candidates.get(0).child(0)); + } + } + } + if (equals.isEmpty()) { + return null; + } else { + List newConjuncts = Lists.newArrayList(equals); + for (Expression conj : conjuncts) { + if (isCandidate(conj)) { + conj = normalizeCandidate((ComparisonPredicate) conj); + if (equalsKey.contains(conj.child(0))) { + continue; + } + } + newConjuncts.add(conj); + } + return ExpressionUtils.and(newConjuncts); + } + } + + // A >= a + // A <= a + // A is expr, a is literal + private static boolean isCandidate(Expression expr) { + if (expr instanceof GreaterThanEqual || expr instanceof LessThanEqual) { + return expr.child(0) instanceof Literal && !(expr.child(1) instanceof Literal) + || expr.child(1) instanceof Literal && !(expr.child(0) instanceof Literal); + } + return false; + } + + private static Expression normalizeCandidate(ComparisonPredicate expr) { + if (expr.child(1) instanceof Literal) { + return expr; + } else { + return expr.withChildren(expr.child(1), expr.child(0)); + } + } + + private static boolean greaterEqualAndLessEqual(ComparisonPredicate cmp1, ComparisonPredicate cmp2) { + return cmp1 instanceof GreaterThanEqual && cmp2 instanceof LessThanEqual + || (cmp1 instanceof LessThanEqual && cmp2 instanceof GreaterThanEqual); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java index fac1a7f82d2cfb..ed783aa3d5a9b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java @@ -55,7 +55,7 @@ public class PartitionPruner extends DefaultExpressionRewriter { /** Different type of table may have different partition prune behavior. */ public enum PartitionTableType { OLAP, - HIVE + EXTERNAL } private PartitionPruner(List partitions, Expression partitionPredicate) { 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 6cf243bbfeec7b..094882294fa6db 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 @@ -47,6 +47,7 @@ import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregatePhase; 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.agg.Max; @@ -294,31 +295,37 @@ public List buildRules() { RuleType.ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT.build( basePattern .when(agg -> agg.getDistinctArguments().isEmpty()) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.ONE)) .thenApplyMulti(ctx -> onePhaseAggregateWithoutDistinct(ctx.root, ctx.connectContext)) ), RuleType.TWO_PHASE_AGGREGATE_WITHOUT_DISTINCT.build( basePattern .when(agg -> agg.getDistinctArguments().isEmpty()) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) .thenApplyMulti(ctx -> twoPhaseAggregateWithoutDistinct(ctx.root, ctx.connectContext)) ), // RuleType.TWO_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI.build( // basePattern // .when(this::containsCountDistinctMultiExpr) + // .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) // .thenApplyMulti(ctx -> twoPhaseAggregateWithCountDistinctMulti(ctx.root, ctx.cascadesContext)) // ), RuleType.THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI.build( basePattern .when(this::containsCountDistinctMultiExpr) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.THREE)) .thenApplyMulti(ctx -> threePhaseAggregateWithCountDistinctMulti(ctx.root, ctx.cascadesContext)) ), RuleType.ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI.build( basePattern .when(agg -> agg.getDistinctArguments().size() == 1 && couldConvertToMulti(agg)) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.ONE)) .thenApplyMulti(ctx -> onePhaseAggregateWithMultiDistinct(ctx.root, ctx.connectContext)) ), RuleType.TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI.build( basePattern .when(agg -> agg.getDistinctArguments().size() == 1 && couldConvertToMulti(agg)) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) .thenApplyMulti(ctx -> twoPhaseAggregateWithMultiDistinct(ctx.root, ctx.connectContext)) ), RuleType.TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT.build( @@ -326,17 +333,20 @@ public List buildRules() { .when(agg -> agg.getDistinctArguments().size() > 1 && !containsCountDistinctMultiExpr(agg) && couldConvertToMulti(agg)) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) .thenApplyMulti(ctx -> twoPhaseAggregateWithMultiDistinct(ctx.root, ctx.connectContext)) ), // RuleType.TWO_PHASE_AGGREGATE_WITH_DISTINCT.build( // basePattern // .when(agg -> agg.getDistinctArguments().size() == 1) + // .when(agg -> agg.supportAggregatePhase(AggregatePhase.TWO)) // .thenApplyMulti(ctx -> twoPhaseAggregateWithDistinct(ctx.root, ctx.connectContext)) // ), RuleType.THREE_PHASE_AGGREGATE_WITH_DISTINCT.build( basePattern .when(agg -> agg.getDistinctArguments().size() == 1) - .whenNot(agg -> agg.mustUseMultiDistinctAgg()) + .whenNot(agg -> agg.mustUseMultiDistinctAgg()) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.THREE)) .thenApplyMulti(ctx -> threePhaseAggregateWithDistinct(ctx.root, ctx.connectContext)) ), /* @@ -361,6 +371,7 @@ && couldConvertToMulti(agg)) .when(agg -> agg.getDistinctArguments().size() == 1) .when(agg -> agg.getGroupByExpressions().isEmpty()) .whenNot(agg -> agg.mustUseMultiDistinctAgg()) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.FOUR)) .thenApplyMulti(ctx -> { Function, RequireProperties> secondPhaseRequireDistinctHash = groupByAndDistinct -> RequireProperties.of( @@ -408,6 +419,7 @@ && couldConvertToMulti(agg)) } return couldConvertToMulti(agg); }) + .when(agg -> agg.supportAggregatePhase(AggregatePhase.FOUR)) .thenApplyMulti(ctx -> { Function, RequireProperties> secondPhaseRequireGroupByAndDistinctHash = groupByAndDistinct -> RequireProperties.of( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java index 74609694431e33..ebef2ecea21207 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java @@ -49,9 +49,13 @@ public class CheckPrivileges extends ColumnPruning { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { + // Only enter once, if repeated, the permissions of the table in the view will be checked + if (jobContext.getCascadesContext().getStatementContext().isPrivChecked()) { + return plan; + } this.jobContext = jobContext; super.rewriteRoot(plan, jobContext); - + jobContext.getCascadesContext().getStatementContext().setPrivChecked(true); // don't rewrite plan return plan; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java new file mode 100644 index 00000000000000..4cb39c2a9341ae --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniform.java @@ -0,0 +1,148 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.jobs.JobContext; +import org.apache.doris.nereids.properties.DataTrait; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.CTEId; +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.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * +--aggregate(group by a, b output a#0 ,b#1, max(c) as max(c)#2) + * (a is uniform and not null: e.g. a is projection 2 as a in logicalProject) + * -> + * +--aggregate(group by b output b#1, any_value(a#0) as a#3, max(c)#2) + * if output any_value(a#0) as a#0, the uniqueness of ExprId #0 is violated, because #0 is both any_value(a#0) and a#0 + * error will occurs in other module(e.g. mv rewrite). + * As a result, new aggregate outputs #3 instead of #0, but upper plan refer slot #0, + * therefore, all references to #0 in the upper plan need to be changed to #3. + * use ExprIdRewriter to do this ExprId rewrite, and use CustomRewriter to rewrite upward。 + * */ +public class EliminateGroupByKeyByUniform extends DefaultPlanRewriter> implements CustomRewriter { + private ExprIdRewriter exprIdReplacer; + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + Optional cteId = jobContext.getCascadesContext().getCurrentTree(); + if (cteId.isPresent()) { + return plan; + } + Map replaceMap = new HashMap<>(); + ExprIdRewriter.ReplaceRule replaceRule = new ExprIdRewriter.ReplaceRule(replaceMap); + exprIdReplacer = new ExprIdRewriter(replaceRule, jobContext); + return plan.accept(this, replaceMap); + } + + @Override + public Plan visit(Plan plan, Map replaceMap) { + plan = visitChildren(this, plan, replaceMap); + plan = exprIdReplacer.rewriteExpr(plan, replaceMap); + return plan; + } + + @Override + public Plan visitLogicalAggregate(LogicalAggregate aggregate, Map replaceMap) { + aggregate = visitChildren(this, aggregate, replaceMap); + aggregate = (LogicalAggregate) exprIdReplacer.rewriteExpr(aggregate, replaceMap); + + if (aggregate.getGroupByExpressions().isEmpty() || aggregate.getSourceRepeat().isPresent()) { + return aggregate; + } + DataTrait aggChildTrait = aggregate.child().getLogicalProperties().getTrait(); + // Get the Group by column of agg. If there is a uniform one, delete the group by key. + Set removedExpression = new LinkedHashSet<>(); + List newGroupBy = new ArrayList<>(); + for (Expression groupBy : aggregate.getGroupByExpressions()) { + if (!(groupBy instanceof Slot)) { + newGroupBy.add(groupBy); + continue; + } + if (aggChildTrait.isUniformAndNotNull((Slot) groupBy)) { + removedExpression.add(groupBy); + } else { + newGroupBy.add(groupBy); + } + } + if (removedExpression.isEmpty()) { + return aggregate; + } + // when newGroupBy is empty, need retain one expr in group by, otherwise the result may be wrong in empty table + if (newGroupBy.isEmpty()) { + Expression expr = removedExpression.iterator().next(); + newGroupBy.add(expr); + removedExpression.remove(expr); + } + if (removedExpression.isEmpty()) { + return aggregate; + } + List newOutputs = new ArrayList<>(); + // If this output appears in the removedExpression column, replace it with any_value + for (NamedExpression output : aggregate.getOutputExpressions()) { + if (output instanceof Slot) { + if (removedExpression.contains(output)) { + Alias alias = new Alias(new AnyValue(false, output), output.getName()); + newOutputs.add(alias); + replaceMap.put(output.getExprId(), alias.getExprId()); + } else { + newOutputs.add(output); + } + } else if (output instanceof Alias) { + if (removedExpression.contains(output.child(0))) { + newOutputs.add(new Alias( + new AnyValue(false, output.child(0)), output.getName())); + } else { + newOutputs.add(output); + } + } else { + newOutputs.add(output); + } + } + + // Adjust the order of this new output so that aggregate functions are placed at the back + // and non-aggregated functions are placed at the front. + List aggFuncs = new ArrayList<>(); + List orderOutput = new ArrayList<>(); + for (NamedExpression output : newOutputs) { + if (output.anyMatch(e -> e instanceof AggregateFunction)) { + aggFuncs.add(output); + } else { + orderOutput.add(output); + } + } + orderOutput.addAll(aggFuncs); + return aggregate.withGroupByAndOutput(newGroupBy, orderOutput); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java new file mode 100644 index 00000000000000..60c9da4bc6eec5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExprIdRewriter.java @@ -0,0 +1,284 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.jobs.JobContext; +import org.apache.doris.nereids.pattern.MatchingContext; +import org.apache.doris.nereids.pattern.Pattern; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher; +import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory; +import org.apache.doris.nereids.rules.expression.ExpressionRewrite; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.ExpressionRuleExecutor; +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.OrderExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; + +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** replace SlotReference ExprId in logical plans */ +public class ExprIdRewriter extends ExpressionRewrite { + private final List rules; + private final JobContext jobContext; + + public ExprIdRewriter(ReplaceRule replaceRule, JobContext jobContext) { + super(new ExpressionRuleExecutor(ImmutableList.of(bottomUp(replaceRule)))); + rules = buildRules(); + this.jobContext = jobContext; + } + + @Override + public List buildRules() { + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(super.buildRules()); + builder.addAll(ImmutableList.of( + new LogicalPartitionTopNExpressionRewrite().build(), + new LogicalTopNExpressionRewrite().build(), + new LogicalSetOperationRewrite().build(), + new LogicalWindowRewrite().build(), + new LogicalResultSinkRewrite().build(), + new LogicalFileSinkRewrite().build(), + new LogicalHiveTableSinkRewrite().build(), + new LogicalIcebergTableSinkRewrite().build(), + new LogicalJdbcTableSinkRewrite().build(), + new LogicalOlapTableSinkRewrite().build(), + new LogicalDeferMaterializeResultSinkRewrite().build() + )); + return builder.build(); + } + + /**rewriteExpr*/ + public Plan rewriteExpr(Plan plan, Map replaceMap) { + if (replaceMap.isEmpty()) { + return plan; + } + for (Rule rule : rules) { + Pattern pattern = (Pattern) rule.getPattern(); + if (pattern.matchPlanTree(plan)) { + List newPlans = rule.transform(plan, jobContext.getCascadesContext()); + Plan newPlan = newPlans.get(0); + if (!newPlan.deepEquals(plan)) { + return newPlan; + } + } + } + return plan; + } + + /** + * Iteratively rewrites IDs using the replaceMap: + * 1. For a given SlotReference with initial ID, retrieve the corresponding value ID from the replaceMap. + * 2. If the value ID exists within the replaceMap, continue the lookup process using the value ID + * until it no longer appears in the replaceMap. + * 3. return SlotReference final value ID as the result of the rewrite. + * e.g. replaceMap:{0:3, 1:6, 6:7} + * SlotReference:a#0 -> a#3, a#1 -> a#7 + * */ + public static class ReplaceRule implements ExpressionPatternRuleFactory { + private final Map replaceMap; + + public ReplaceRule(Map replaceMap) { + this.replaceMap = replaceMap; + } + + @Override + public List> buildRules() { + return ImmutableList.of( + matchesType(SlotReference.class).thenApply(ctx -> { + Slot slot = ctx.expr; + if (replaceMap.containsKey(slot.getExprId())) { + ExprId newId = replaceMap.get(slot.getExprId()); + while (replaceMap.containsKey(newId)) { + newId = replaceMap.get(newId); + } + return slot.withExprId(newId); + } + return slot; + }) + ); + } + } + + private class LogicalResultSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalResultSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalFileSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalFileSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalHiveTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalHiveTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalIcebergTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalIcebergTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalJdbcTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalJdbcTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalOlapTableSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalOlapTableSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalDeferMaterializeResultSinkRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalDeferMaterializeResultSink().thenApply(ExprIdRewriter.this::applyRewrite) + .toRule(RuleType.REWRITE_SINK_EXPRESSION); + } + } + + private class LogicalSetOperationRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalSetOperation().thenApply(ctx -> { + LogicalSetOperation setOperation = ctx.root; + List> slotsList = setOperation.getRegularChildrenOutputs(); + List> newSlotsList = new ArrayList<>(); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + for (List slots : slotsList) { + List newSlots = rewriteAll(slots, rewriter, context); + newSlotsList.add(newSlots); + } + if (newSlotsList.equals(slotsList)) { + return setOperation; + } + return setOperation.withChildrenAndTheirOutputs(setOperation.children(), newSlotsList); + }) + .toRule(RuleType.REWRITE_SET_OPERATION_EXPRESSION); + } + } + + private class LogicalWindowRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalWindow().thenApply(ctx -> { + LogicalWindow window = ctx.root; + List windowExpressions = window.getWindowExpressions(); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List newWindowExpressions = rewriteAll(windowExpressions, rewriter, context); + if (newWindowExpressions.equals(windowExpressions)) { + return window; + } + return window.withExpressionsAndChild(newWindowExpressions, window.child()); + }) + .toRule(RuleType.REWRITE_WINDOW_EXPRESSION); + } + } + + private class LogicalTopNExpressionRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalTopN().thenApply(ctx -> { + LogicalTopN topN = ctx.root; + List orderKeys = topN.getOrderKeys(); + ImmutableList.Builder rewrittenOrderKeys + = ImmutableList.builderWithExpectedSize(orderKeys.size()); + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + boolean changed = false; + for (OrderKey k : orderKeys) { + Expression expression = rewriter.rewrite(k.getExpr(), context); + changed |= expression != k.getExpr(); + rewrittenOrderKeys.add(new OrderKey(expression, k.isAsc(), k.isNullFirst())); + } + return changed ? topN.withOrderKeys(rewrittenOrderKeys.build()) : topN; + }).toRule(RuleType.REWRITE_TOPN_EXPRESSION); + } + } + + private class LogicalPartitionTopNExpressionRewrite extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalPartitionTopN().thenApply(ctx -> { + LogicalPartitionTopN partitionTopN = ctx.root; + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List newOrderExpressions = new ArrayList<>(); + boolean changed = false; + for (OrderExpression orderExpression : partitionTopN.getOrderKeys()) { + OrderKey orderKey = orderExpression.getOrderKey(); + Expression expr = rewriter.rewrite(orderKey.getExpr(), context); + changed |= expr != orderKey.getExpr(); + OrderKey newOrderKey = new OrderKey(expr, orderKey.isAsc(), orderKey.isNullFirst()); + newOrderExpressions.add(new OrderExpression(newOrderKey)); + } + List newPartitionKeys = rewriteAll(partitionTopN.getPartitionKeys(), rewriter, context); + if (!newPartitionKeys.equals(partitionTopN.getPartitionKeys())) { + changed = true; + } + if (!changed) { + return partitionTopN; + } + return partitionTopN.withPartitionKeysAndOrderKeys(newPartitionKeys, newOrderExpressions); + }).toRule(RuleType.REWRITE_PARTITION_TOPN_EXPRESSION); + } + } + + private LogicalSink applyRewrite(MatchingContext> ctx) { + LogicalSink sink = ctx.root; + ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext); + List outputExprs = sink.getOutputExprs(); + List newOutputExprs = rewriteAll(outputExprs, rewriter, context); + if (outputExprs.equals(newOutputExprs)) { + return sink; + } + return sink.withOutputExprs(newOutputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java index 5465211c73e793..c7b8f452afbe66 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneEmptyPartition.java @@ -36,11 +36,16 @@ public Rule build() { return logicalOlapScan().thenApply(ctx -> { LogicalOlapScan scan = ctx.root; OlapTable table = scan.getTable(); - List ids = table.selectNonEmptyPartitionIds(scan.getSelectedPartitionIds()); + List partitionIdsToPrune = scan.getSelectedPartitionIds(); + List ids = table.selectNonEmptyPartitionIds(partitionIdsToPrune); if (ids.isEmpty()) { return new LogicalEmptyRelation(ConnectContext.get().getStatementContext().getNextRelationId(), scan.getOutput()); } + if (partitionIdsToPrune.equals(ids)) { + // Not Prune actually, return directly + return null; + } return scan.withSelectedPartitionIds(ids); }).toRule(RuleType.PRUNE_EMPTY_PARTITION); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index 4bbb0a8aa76270..ba8b270d1f397d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -59,7 +59,7 @@ public Rule build() { ExternalTable tbl = scan.getTable(); SelectedPartitions selectedPartitions; - if (tbl.supportPartitionPruned()) { + if (tbl.supportInternalPartitionPruned()) { selectedPartitions = pruneExternalPartitions(tbl, filter, scan, ctx.cascadesContext); } else { // set isPruned so that it won't go pass the partition prune again @@ -91,7 +91,7 @@ private SelectedPartitions pruneExternalPartitions(ExternalTable externalTable, Map nameToPartitionItem = scan.getSelectedPartitions().selectedPartitions; List prunedPartitions = new ArrayList<>(PartitionPruner.prune( - partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.HIVE)); + partitionSlots, filter.getPredicate(), nameToPartitionItem, ctx, PartitionTableType.EXTERNAL)); for (String name : prunedPartitions) { selectedPartitionItems.put(name, nameToPartitionItem.get(name)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java index 4b86b7a3706f0e..fc212550b6f364 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTablet.java @@ -53,7 +53,7 @@ public Rule build() { LogicalOlapScan olapScan = filter.child(); OlapTable table = olapScan.getTable(); Builder selectedTabletIdsBuilder = ImmutableList.builder(); - if (olapScan.getSelectedTabletIds().isEmpty()) { + if (olapScan.getManuallySpecifiedTabletIds().isEmpty()) { for (Long id : olapScan.getSelectedPartitionIds()) { Partition partition = table.getPartition(id); MaterializedIndex index = partition.getIndex(olapScan.getSelectedIndexId()); @@ -64,10 +64,10 @@ public Rule build() { partition.getDistributionInfo())); } } else { - selectedTabletIdsBuilder.addAll(olapScan.getSelectedTabletIds()); + selectedTabletIdsBuilder.addAll(olapScan.getManuallySpecifiedTabletIds()); } List selectedTabletIds = selectedTabletIdsBuilder.build(); - if (new HashSet<>(selectedTabletIds).equals(new HashSet<>(olapScan.getSelectedTabletIds()))) { + if (new HashSet<>(selectedTabletIds).equals(new HashSet<>(olapScan.getManuallySpecifiedTabletIds()))) { return null; } return filter.withChildren(olapScan.withSelectedTabletIds(selectedTabletIds)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java index f32bf8ea91b355..c5d3d0fb49a0a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggThroughJoinOneSide.java @@ -36,6 +36,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.HashMap; @@ -74,8 +75,8 @@ public List buildRules() { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream() .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum - || (f instanceof Count && !((Count) f).isCountStar())) && !f.isDistinct() - && f.child(0) instanceof Slot); + || f instanceof Count && !f.isDistinct() + && (f.children().isEmpty() || f.child(0) instanceof Slot))); }) .thenApply(ctx -> { Set enableNereidsRules = ctx.cascadesContext.getConnectContext() @@ -88,15 +89,16 @@ public List buildRules() { }) .toRule(RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE), logicalAggregate(logicalProject(innerLogicalJoin())) - .when(agg -> agg.child().isAllSlots()) - .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) - .whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate)) + // .when(agg -> agg.child().isAllSlots()) + // .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) + .whenNot(agg -> agg.child() + .child(0).children().stream().anyMatch(p -> p instanceof LogicalAggregate)) .when(agg -> { Set funcs = agg.getAggregateFunctions(); return !funcs.isEmpty() && funcs.stream() .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum - || (f instanceof Count && (!((Count) f).isCountStar()))) && !f.isDistinct() - && f.child(0) instanceof Slot); + || f instanceof Count) && !f.isDistinct() + && (f.children().isEmpty() || f.child(0) instanceof Slot)); }) .thenApply(ctx -> { Set enableNereidsRules = ctx.cascadesContext.getConnectContext() @@ -118,23 +120,6 @@ public static LogicalAggregate pushMinMaxSumCount(LogicalAggregate join, List projects) { List leftOutput = join.left().getOutput(); List rightOutput = join.right().getOutput(); - - List leftFuncs = new ArrayList<>(); - List rightFuncs = new ArrayList<>(); - for (AggregateFunction func : agg.getAggregateFunctions()) { - Slot slot = (Slot) func.child(0); - if (leftOutput.contains(slot)) { - leftFuncs.add(func); - } else if (rightOutput.contains(slot)) { - rightFuncs.add(func); - } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); - } - } - if (leftFuncs.isEmpty() && rightFuncs.isEmpty()) { - return null; - } - Set leftGroupBy = new HashSet<>(); Set rightGroupBy = new HashSet<>(); for (Expression e : agg.getGroupByExpressions()) { @@ -144,18 +129,71 @@ public static LogicalAggregate pushMinMaxSumCount(LogicalAggregate inputForAliasSet = proj.getInputSlots(); + for (Slot aliasInputSlot : inputForAliasSet) { + if (leftOutput.contains(aliasInputSlot)) { + leftGroupBy.add(aliasInputSlot); + } else if (rightOutput.contains(aliasInputSlot)) { + rightGroupBy.add(aliasInputSlot); + } else { + return null; + } + } + break; + } + } + } } } - join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> { - if (leftOutput.contains(slot)) { - leftGroupBy.add(slot); - } else if (rightOutput.contains(slot)) { - rightGroupBy.add(slot); + + List leftFuncs = new ArrayList<>(); + List rightFuncs = new ArrayList<>(); + Count countStar = null; + Count rewrittenCountStar = null; + for (AggregateFunction func : agg.getAggregateFunctions()) { + if (func instanceof Count && ((Count) func).isCountStar()) { + countStar = (Count) func; + } else { + Slot slot = (Slot) func.child(0); + if (leftOutput.contains(slot)) { + leftFuncs.add(func); + } else if (rightOutput.contains(slot)) { + rightFuncs.add(func); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + } + } + // rewrite count(*) to count(A), where A is slot from left/right group by key + if (countStar != null) { + if (!leftGroupBy.isEmpty()) { + rewrittenCountStar = (Count) countStar.withChildren(leftGroupBy.iterator().next()); + leftFuncs.add(rewrittenCountStar); + } else if (!rightGroupBy.isEmpty()) { + rewrittenCountStar = (Count) countStar.withChildren(rightGroupBy.iterator().next()); + rightFuncs.add(rewrittenCountStar); } else { - throw new IllegalStateException("Slot " + slot + " not found in join output"); + return null; + } + } + for (Expression condition : join.getHashJoinConjuncts()) { + for (Slot joinConditionSlot : condition.getInputSlots()) { + if (leftOutput.contains(joinConditionSlot)) { + leftGroupBy.add(joinConditionSlot); + } else if (rightOutput.contains(joinConditionSlot)) { + rightGroupBy.add(joinConditionSlot); + } else { + // apply failed + return null; + } } - })); + } Plan left = join.left(); Plan right = join.right(); @@ -196,6 +234,10 @@ public static LogicalAggregate pushMinMaxSumCount(LogicalAggregate pushMinMaxSumCount(LogicalAggregate newProjections = Lists.newArrayList(); + newProjections.addAll(project.getProjects()); + Set leftDifference = new HashSet(left.getOutput()); + leftDifference.removeAll(project.getProjects()); + newProjections.addAll(leftDifference); + Set rightDifference = new HashSet(right.getOutput()); + rightDifference.removeAll(project.getProjects()); + newProjections.addAll(rightDifference); + newAggChild = ((LogicalProject) agg.child()).withProjectsAndChild(newProjections, newJoin); + } + return agg.withAggOutputChild(newOutputExprs, newAggChild); } private static Expression replaceAggFunc(AggregateFunction func, Slot inputSlot) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java new file mode 100644 index 00000000000000..3f9ad609744e21 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java @@ -0,0 +1,176 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +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.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Min; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Push down agg function with distinct through join on only one side. + */ +public class PushDownAggWithDistinctThroughJoinOneSide implements RewriteRuleFactory { + @Override + public List buildRules() { + return ImmutableList.of( + logicalAggregate(logicalProject(innerLogicalJoin())) + .when(agg -> agg.child().isAllSlots()) + .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) + .when(agg -> !agg.isGenerated()) + .whenNot(agg -> agg.getAggregateFunctions().isEmpty()) + .whenNot(agg -> agg.child() + .child(0).children().stream().anyMatch(p -> p instanceof LogicalAggregate)) + .when(agg -> { + Set funcs = agg.getAggregateFunctions(); + if (funcs.size() > 1) { + return false; + } else { + return funcs.stream() + .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum + || f instanceof Count) && f.isDistinct() + && f.child(0) instanceof Slot); + } + }) + .thenApply(ctx -> { + LogicalAggregate>> agg = ctx.root; + return pushDownAggWithDistinct(agg, agg.child().child(), agg.child().getProjects()); + }) + .toRule(RuleType.PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE) + ); + } + + private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate agg, + LogicalJoin join, List projects) { + Plan leftJoin = join.left(); + Plan rightJoin = join.right(); + List leftJoinOutput = leftJoin.getOutput(); + List rightJoinOutput = rightJoin.getOutput(); + + List leftFuncs = new ArrayList<>(); + List rightFuncs = new ArrayList<>(); + Set leftFuncSlotSet = new HashSet<>(); + Set rightFuncSlotSet = new HashSet<>(); + Set newAggOverJoinGroupByKeys = new HashSet<>(); + for (AggregateFunction func : agg.getAggregateFunctions()) { + Slot slot = (Slot) func.child(0); + newAggOverJoinGroupByKeys.add(slot); + if (leftJoinOutput.contains(slot)) { + leftFuncs.add(func); + leftFuncSlotSet.add(slot); + } else if (rightJoinOutput.contains(slot)) { + rightFuncs.add(func); + rightFuncSlotSet.add(slot); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + } + boolean isLeftSideAggDistinct = !leftFuncs.isEmpty() && rightFuncs.isEmpty(); + boolean isRightSideAggDistinct = leftFuncs.isEmpty() && !rightFuncs.isEmpty(); + if (!isLeftSideAggDistinct && !isRightSideAggDistinct) { + return null; + } + + Set leftPushDownGroupBy = new HashSet<>(); + Set rightPushDownGroupBy = new HashSet<>(); + for (Expression e : agg.getGroupByExpressions()) { + Slot slot = (Slot) e; + newAggOverJoinGroupByKeys.add(slot); + if (leftJoinOutput.contains(slot)) { + leftPushDownGroupBy.add(slot); + } else if (rightJoinOutput.contains(slot)) { + rightPushDownGroupBy.add(slot); + } else { + return null; + } + } + join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> { + if (leftJoinOutput.contains(slot)) { + leftPushDownGroupBy.add(slot); + } else if (rightJoinOutput.contains(slot)) { + rightPushDownGroupBy.add(slot); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + })); + + if (isLeftSideAggDistinct) { + leftPushDownGroupBy.add((Slot) leftFuncs.get(0).child(0)); + Builder leftAggOutputBuilder = ImmutableList.builder() + .addAll(leftPushDownGroupBy); + leftJoin = new LogicalAggregate<>(ImmutableList.copyOf(leftPushDownGroupBy), + leftAggOutputBuilder.build(), join.left()); + } else { + rightPushDownGroupBy.add((Slot) rightFuncs.get(0).child(0)); + Builder rightAggOutputBuilder = ImmutableList.builder() + .addAll(rightPushDownGroupBy); + rightJoin = new LogicalAggregate<>(ImmutableList.copyOf(rightPushDownGroupBy), + rightAggOutputBuilder.build(), join.right()); + } + + Preconditions.checkState(leftJoin != join.left() || rightJoin != join.right(), + "not pushing down aggr with distinct through join on single side successfully"); + Plan newJoin = join.withChildren(leftJoin, rightJoin); + LogicalAggregate newAggOverJoin = agg.withChildGroupByAndOutput( + ImmutableList.copyOf(newAggOverJoinGroupByKeys), projects, newJoin); + + List newOutputExprs = new ArrayList<>(); + for (NamedExpression ne : agg.getOutputExpressions()) { + if (ne instanceof Alias && ((Alias) ne).child() instanceof AggregateFunction) { + AggregateFunction func = (AggregateFunction) ((Alias) ne).child(); + Slot slot = (Slot) func.child(0); + if (leftFuncSlotSet.contains(slot) || rightFuncSlotSet.contains(slot)) { + Expression newFunc = discardDistinct(func); + newOutputExprs.add((NamedExpression) ne.withChildren(newFunc)); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + } else { + newOutputExprs.add(ne); + } + } + return agg.withAggOutputChild(newOutputExprs, newAggOverJoin); + } + + private static Expression discardDistinct(AggregateFunction func) { + Preconditions.checkState(func.isDistinct(), "current aggregation function is not distinct"); + Set aggChild = Sets.newLinkedHashSet(func.children()); + return func.withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 2c4fd340bf585e..0249a94d67b7fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -18,11 +18,18 @@ package org.apache.doris.nereids.stats; import org.apache.doris.analysis.IntLiteral; +import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ListPartitionItem; import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.RangePartitionItem; import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; @@ -143,11 +150,13 @@ import org.apache.doris.statistics.Statistics; import org.apache.doris.statistics.StatisticsBuilder; import org.apache.doris.statistics.TableStatsMeta; +import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Range; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -544,6 +553,9 @@ private Statistics computeOlapScan(OlapScan olapScan) { }); for (SlotReference slot : visibleOutputSlots) { ColumnStatistic cache = getColumnStatsFromPartitionCache(olapScan, slot, selectedPartitionNames); + if (slot.getColumn().isPresent()) { + cache = updateMinMaxForPartitionKey(olapTable, selectedPartitionNames, slot, cache); + } ColumnStatisticBuilder colStatsBuilder = new ColumnStatisticBuilder(cache, selectedPartitionsRowCount); colStatsBuilder.normalizeAvgSizeByte(slot); @@ -576,6 +588,125 @@ private Statistics computeOlapScan(OlapScan olapScan) { return builder.build(); } + private ColumnStatistic updateMinMaxForPartitionKey(OlapTable olapTable, + List selectedPartitionNames, + SlotReference slot, ColumnStatistic cache) { + if (olapTable.getPartitionType() == PartitionType.LIST) { + cache = updateMinMaxForListPartitionKey(olapTable, selectedPartitionNames, slot, cache); + } else if (olapTable.getPartitionType() == PartitionType.RANGE) { + cache = updateMinMaxForTheFirstRangePartitionKey(olapTable, selectedPartitionNames, slot, cache); + } + return cache; + } + + private double convertLegacyLiteralToDouble(LiteralExpr literal) throws AnalysisException { + return StatisticsUtil.convertToDouble(literal.getType(), literal.getStringValue()); + } + + private ColumnStatistic updateMinMaxForListPartitionKey(OlapTable olapTable, + List selectedPartitionNames, + SlotReference slot, ColumnStatistic cache) { + int partitionColumnIdx = olapTable.getPartitionColumns().indexOf(slot.getColumn().get()); + if (partitionColumnIdx != -1) { + try { + LiteralExpr minExpr = null; + LiteralExpr maxExpr = null; + double minValue = 0; + double maxValue = 0; + for (String selectedPartitionName : selectedPartitionNames) { + PartitionItem item = olapTable.getPartitionItemOrAnalysisException( + selectedPartitionName); + if (item instanceof ListPartitionItem) { + ListPartitionItem lp = (ListPartitionItem) item; + for (PartitionKey key : lp.getItems()) { + if (minExpr == null) { + minExpr = key.getKeys().get(partitionColumnIdx); + minValue = convertLegacyLiteralToDouble(minExpr); + maxExpr = key.getKeys().get(partitionColumnIdx); + maxValue = convertLegacyLiteralToDouble(maxExpr); + } else { + double current = convertLegacyLiteralToDouble(key.getKeys().get(partitionColumnIdx)); + if (current > maxValue) { + maxValue = current; + maxExpr = key.getKeys().get(partitionColumnIdx); + } else if (current < minValue) { + minValue = current; + minExpr = key.getKeys().get(partitionColumnIdx); + } + } + } + } + } + if (minExpr != null) { + cache = new ColumnStatisticBuilder(cache) + .setMinExpr(minExpr) + .setMinValue(minValue) + .setMaxExpr(maxExpr) + .setMaxValue(maxValue) + .build(); + } + } catch (AnalysisException e) { + LOG.debug(e.getMessage()); + } + } + return cache; + } + + private ColumnStatistic updateMinMaxForTheFirstRangePartitionKey(OlapTable olapTable, + List selectedPartitionNames, + SlotReference slot, ColumnStatistic cache) { + int partitionColumnIdx = olapTable.getPartitionColumns().indexOf(slot.getColumn().get()); + // for multi partition keys, only the first partition key need to adjust min/max + if (partitionColumnIdx == 0) { + // update partition column min/max by partition info + try { + LiteralExpr minExpr = null; + LiteralExpr maxExpr = null; + double minValue = 0; + double maxValue = 0; + for (String selectedPartitionName : selectedPartitionNames) { + PartitionItem item = olapTable.getPartitionItemOrAnalysisException( + selectedPartitionName); + if (item instanceof RangePartitionItem) { + RangePartitionItem ri = (RangePartitionItem) item; + Range range = ri.getItems(); + PartitionKey upper = range.upperEndpoint(); + PartitionKey lower = range.lowerEndpoint(); + if (maxExpr == null) { + maxExpr = upper.getKeys().get(partitionColumnIdx); + maxValue = convertLegacyLiteralToDouble(maxExpr); + minExpr = lower.getKeys().get(partitionColumnIdx); + minValue = convertLegacyLiteralToDouble(minExpr); + } else { + double currentValue = convertLegacyLiteralToDouble(upper.getKeys() + .get(partitionColumnIdx)); + if (currentValue > maxValue) { + maxValue = currentValue; + maxExpr = upper.getKeys().get(partitionColumnIdx); + } + currentValue = convertLegacyLiteralToDouble(lower.getKeys().get(partitionColumnIdx)); + if (currentValue < minValue) { + minValue = currentValue; + minExpr = lower.getKeys().get(partitionColumnIdx); + } + } + } + } + if (minExpr != null) { + cache = new ColumnStatisticBuilder(cache) + .setMinExpr(minExpr) + .setMinValue(minValue) + .setMaxExpr(maxExpr) + .setMaxValue(maxValue) + .build(); + } + } catch (AnalysisException e) { + LOG.debug(e.getMessage()); + } + } + return cache; + } + @Override public Statistics visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { return computeOlapScan(olapScan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java index 6b12f9cd642992..8fda4d4b020f2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AlwaysNotNullable.java @@ -17,9 +17,6 @@ package org.apache.doris.nereids.trees.expressions.functions; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Expression; - /** * nullable is always false. * @@ -30,10 +27,4 @@ public interface AlwaysNotNullable extends ComputeNullable { default boolean nullable() { return false; } - - // return value of this function if the input data is empty. - // for example, count(*) of empty table is 0; - default Expression resultForEmptyInput() { - throw new AnalysisException("should implement resultForEmptyInput() for " + this.getClass()); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java index 58b9d0274ddfc5..90df2f531da3fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java @@ -131,6 +131,10 @@ public String toString() { return getName() + "(" + (distinct ? "DISTINCT " : "") + args + ")"; } + public boolean supportAggregatePhase(AggregatePhase aggregatePhase) { + return true; + } + public List getDistinctArguments() { return distinct ? getArguments() : ImmutableList.of(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregatePhase.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregatePhase.java new file mode 100644 index 00000000000000..9115e5890ad66c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregatePhase.java @@ -0,0 +1,23 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.agg; + +/** AggregatePhase */ +public enum AggregatePhase { + ONE, TWO, THREE, FOUR +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java index bc91207e31f3a6..6218569261a775 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/ArrayAgg.java @@ -19,7 +19,6 @@ 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.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -37,8 +36,8 @@ /** * AggregateFunction 'array_agg'. */ -public class ArrayAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class ArrayAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java index eaf766b908d7af..a0eaf88efaf13b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapAgg.java @@ -19,7 +19,6 @@ 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.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -37,8 +36,8 @@ /** * AggregateFunction 'bitmap_agg'. */ -public class BitmapAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class BitmapAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BigIntType.INSTANCE), FunctionSignature.ret(BitmapType.INSTANCE).args(IntegerType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java index 1b7d2d3c3cddec..80a9887d325634 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapIntersect.java @@ -19,7 +19,6 @@ 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.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -35,8 +34,8 @@ /** * AggregateFunction 'bitmap_intersect'. This class is generated by GenerateFunction. */ -public class BitmapIntersect extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction { +public class BitmapIntersect extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java index cd0756a1c9361f..fce2f4fa70c483 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnion.java @@ -19,7 +19,6 @@ 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.Function; import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; @@ -36,8 +35,8 @@ /** * AggregateFunction 'bitmap_union'. This class is generated by GenerateFunction. */ -public class BitmapUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction, RollUpTrait { +public class BitmapUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BitmapType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java index 593c814f22d159..d1e2d6010a27a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionCount.java @@ -19,7 +19,6 @@ 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.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'bitmap_union_count'. This class is generated by GenerateFunction. */ -public class BitmapUnionCount extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, BitmapFunction, RollUpTrait { +public class BitmapUnionCount extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, BitmapFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java index 2efe1631176c86..af569982c4e62e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BitmapUnionInt.java @@ -20,7 +20,6 @@ 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.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -41,8 +40,8 @@ /** * AggregateFunction 'bitmap_union_int'. This class is generated by GenerateFunction. */ -public class BitmapUnionInt extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class BitmapUnionInt extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(SmallIntType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java index d6cca2d0b90b6d..ed0dd2406211ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectList.java @@ -19,7 +19,6 @@ 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.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -38,8 +37,8 @@ /** * AggregateFunction 'collect_list'. This class is generated by GenerateFunction. */ -public class CollectList extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class CollectList extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java index d9e7e7227c6f86..2aba485fc7daaa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CollectSet.java @@ -19,7 +19,6 @@ 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.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -38,8 +37,8 @@ /** * AggregateFunction 'collect_set'. This class is generated by GenerateFunction. */ -public class CollectSet extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class CollectSet extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(new FollowToAnyDataType(0))).args(new AnyDataType(0)), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java index 10874d47ee3df9..e86e90974da1bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java @@ -20,7 +20,6 @@ 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.Function; import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; @@ -37,8 +36,8 @@ import java.util.List; /** count agg function. */ -public class Count extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic, RollUpTrait { +public class Count extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, SupportWindowAnalytic, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( // count(*) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java index 2a4ee7be3f49c2..f956854d6871bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CountByEnum.java @@ -19,7 +19,6 @@ 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.StringLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -31,7 +30,7 @@ import java.util.List; /** count_by_enum agg function. */ -public class CountByEnum extends AggregateFunction implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class CountByEnum extends NotNullableAggregateFunction implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java index 0720d6838bb1b6..1cc0db1bd5ed5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/GroupArrayIntersect.java @@ -19,7 +19,6 @@ 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.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'group_array_intersect'. */ -public class GroupArrayIntersect extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class GroupArrayIntersect extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.retArgType(0) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java index 6b0a2759823f3a..827c57facd7498 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Histogram.java @@ -19,7 +19,6 @@ 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.SearchSignature; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'histogram'. This class is generated by GenerateFunction. */ -public class Histogram extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable { +public class Histogram extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java index b81fad270b0cda..8f6224f66b0043 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnion.java @@ -19,7 +19,6 @@ 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.Function; import org.apache.doris.nereids.trees.expressions.functions.scalar.HllEmpty; @@ -36,8 +35,8 @@ /** * AggregateFunction 'hll_union'. This class is generated by GenerateFunction. */ -public class HllUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, HllFunction, RollUpTrait { +public class HllUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, HllFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(HllType.INSTANCE).args(HllType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java index b14b61b5be07e9..b2c15b402aa2c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/HllUnionAgg.java @@ -19,7 +19,6 @@ 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.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'hll_union_agg'. This class is generated by GenerateFunction. */ -public class HllUnionAgg extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, HllFunction, RollUpTrait { +public class HllUnionAgg extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, HllFunction, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(HllType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java index c013b2e8b4c31d..3eed9b9a6e4ddb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/IntersectCount.java @@ -19,7 +19,6 @@ 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.BitmapIntersectFunction; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -37,8 +36,8 @@ /** * AggregateFunction 'intersect_count'. This class is generated by GenerateFunction. */ -public class IntersectCount extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, BitmapIntersectFunction { +public class IntersectCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, BitmapIntersectFunction { public static final List SIGNATURES = DataType.trivialTypes().stream() .map(type -> FunctionSignature.ret(BigIntType.INSTANCE).varArgs(BitmapType.INSTANCE, type, type)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java index 744d4a23a667e5..798b3677ff127d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MapAgg.java @@ -19,7 +19,6 @@ 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.MapLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'map_agg'. This class is generated by GenerateFunction. */ -public class MapAgg extends AggregateFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class MapAgg extends NotNullableAggregateFunction + implements BinaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(MapType.of(new FollowToAnyDataType(0), new FollowToAnyDataType(1))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java index 68d31e3e7bd19d..81768ef48dd960 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctCount.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.trees.expressions.Cast; 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.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -36,8 +35,8 @@ import java.util.List; /** MultiDistinctCount */ -public class MultiDistinctCount extends AggregateFunction - implements AlwaysNotNullable, ExplicitlyCastableSignature, MultiDistinction { +public class MultiDistinctCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, MultiDistinction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).varArgs(AnyDataType.INSTANCE_WITHOUT_INDEX) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java index 2b0eda06b4264a..abd5292e31a233 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/MultiDistinctSum0.java @@ -20,7 +20,6 @@ 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.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -39,8 +38,8 @@ import java.util.List; /** MultiDistinctSum0 */ -public class MultiDistinctSum0 extends AggregateFunction implements UnaryExpression, - ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction, AlwaysNotNullable { +public class MultiDistinctSum0 extends NotNullableAggregateFunction implements UnaryExpression, + ExplicitlyCastableSignature, ComputePrecisionForSum, MultiDistinction { private final boolean mustUseMultiDistinctAgg; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java index ea90bc587911be..c2a04b4fd9931b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Ndv.java @@ -21,7 +21,6 @@ import org.apache.doris.catalog.Type; 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.Function; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; @@ -38,8 +37,8 @@ /** * AggregateFunction 'ndv'. This class is generated by GenserateFunction. */ -public class Ndv extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable, RollUpTrait { +public class Ndv extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(AnyDataType.INSTANCE_WITHOUT_INDEX) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java new file mode 100644 index 00000000000000..f38decf439c2c6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/NotNullableAggregateFunction.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.agg; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; + +import java.util.List; + +/** + * base class for AlwaysNotNullable aggregate function + */ +public abstract class NotNullableAggregateFunction extends AggregateFunction implements AlwaysNotNullable { + protected NotNullableAggregateFunction(String name, Expression ...expressions) { + super(name, false, expressions); + } + + protected NotNullableAggregateFunction(String name, List expressions) { + super(name, false, expressions); + } + + protected NotNullableAggregateFunction(String name, boolean distinct, Expression ...expressions) { + super(name, distinct, expressions); + } + + protected NotNullableAggregateFunction(String name, boolean distinct, List expressions) { + super(name, distinct, expressions); + } + + // return value of this function if the input data is empty. + // for example, count(*) of empty table is 0; + public abstract Expression resultForEmptyInput(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapExprCalculate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapExprCalculate.java new file mode 100644 index 00000000000000..4a91e520055efe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapExprCalculate.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.agg; + +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.scalar.BitmapEmpty; +import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; +import org.apache.doris.nereids.types.BitmapType; +import org.apache.doris.nereids.types.VarcharType; +import org.apache.doris.nereids.types.coercion.CharacterType; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** OrthogonalBitmapExprCalculate */ +public class OrthogonalBitmapExprCalculate extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, ExplicitlyCastableSignature { + + static final List FUNCTION_SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BitmapType.INSTANCE) + .varArgs(BitmapType.INSTANCE, VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT) + ); + + /** + * constructor with 3 arguments. + */ + public OrthogonalBitmapExprCalculate( + Expression bitmap, Expression filterColumn, VarcharLiteral inputString) { + super("orthogonal_bitmap_expr_calculate", ExpressionUtils.mergeArguments(bitmap, filterColumn, inputString)); + } + + /** + * constructor with 3 arguments. + */ + public OrthogonalBitmapExprCalculate(boolean distinct, + Expression bitmap, Expression filterColumn, VarcharLiteral inputString) { + super("orthogonal_bitmap_expr_calculate", distinct, + ExpressionUtils.mergeArguments(bitmap, filterColumn, inputString)); + } + + @Override + public boolean supportAggregatePhase(AggregatePhase aggregatePhase) { + return aggregatePhase == AggregatePhase.TWO; + } + + @Override + public OrthogonalBitmapExprCalculate withDistinctAndChildren(boolean distinct, List children) { + Preconditions.checkArgument(children.size() == 3 + && children.get(2).getDataType() instanceof CharacterType + && children.get(2).getDataType() instanceof VarcharType); + return new OrthogonalBitmapExprCalculate( + distinct, children.get(0), children.get(1), (VarcharLiteral) children.get(2)); + } + + @Override + public List getSignatures() { + return FUNCTION_SIGNATURES; + } + + @Override + public Expression resultForEmptyInput() { + return new BitmapEmpty(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapExprCalculateCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapExprCalculateCount.java new file mode 100644 index 00000000000000..3100cc9ae80b05 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapExprCalculateCount.java @@ -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. + +package org.apache.doris.nereids.trees.expressions.functions.agg; + +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.scalar.BitmapEmpty; +import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.BitmapType; +import org.apache.doris.nereids.types.VarcharType; +import org.apache.doris.nereids.types.coercion.CharacterType; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** OrthogonalBitmapExprCalculateCount */ +public class OrthogonalBitmapExprCalculateCount extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, ExplicitlyCastableSignature { + + static final List FUNCTION_SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE) + .varArgs(BitmapType.INSTANCE, VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT) + ); + + /** + * constructor with 3 arguments. + */ + public OrthogonalBitmapExprCalculateCount( + Expression bitmap, Expression filterColumn, VarcharLiteral inputString) { + super("orthogonal_bitmap_expr_calculate_count", + ExpressionUtils.mergeArguments(bitmap, filterColumn, inputString)); + } + + /** + * constructor with 3 arguments. + */ + public OrthogonalBitmapExprCalculateCount(boolean distinct, + Expression bitmap, Expression filterColumn, VarcharLiteral inputString) { + super("orthogonal_bitmap_expr_calculate_count", distinct, + ExpressionUtils.mergeArguments(bitmap, filterColumn, inputString)); + } + + @Override + public boolean supportAggregatePhase(AggregatePhase aggregatePhase) { + return aggregatePhase == AggregatePhase.TWO; + } + + @Override + public OrthogonalBitmapExprCalculateCount withDistinctAndChildren(boolean distinct, List children) { + Preconditions.checkArgument(children.size() == 3 + && children.get(2).getDataType() instanceof CharacterType + && children.get(2).getDataType() instanceof VarcharType); + return new OrthogonalBitmapExprCalculateCount( + distinct, children.get(0), children.get(1), (VarcharLiteral) children.get(2)); + } + + @Override + public List getSignatures() { + return FUNCTION_SIGNATURES; + } + + @Override + public Expression resultForEmptyInput() { + return new BitmapEmpty(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java index 956f585f0a1e4c..7837ea1bec7569 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersect.java @@ -19,8 +19,8 @@ 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.BitmapIntersectFunction; +import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapEmpty; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BitmapType; import org.apache.doris.nereids.util.ExpressionUtils; @@ -33,8 +33,8 @@ /** * AggregateFunction 'orthogonal_bitmap_intersect'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapIntersect extends AggregateFunction - implements AlwaysNotNullable, OrthogonalBitmapFunction, BitmapIntersectFunction { +public class OrthogonalBitmapIntersect extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, BitmapIntersectFunction { static final List FUNCTION_SIGNATURES = SUPPORTED_TYPES.stream() .map(type -> FunctionSignature.ret(BitmapType.INSTANCE).varArgs(BitmapType.INSTANCE, type, type)) @@ -75,4 +75,9 @@ public OrthogonalBitmapIntersect withDistinctAndChildren(boolean distinct, List< public R accept(ExpressionVisitor visitor, C context) { return visitor.visitOrthogonalBitmapIntersect(this, context); } + + @Override + public Expression resultForEmptyInput() { + return new BitmapEmpty(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java index 82bfa911c26faf..fd282341249e0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapIntersectCount.java @@ -18,8 +18,8 @@ package org.apache.doris.nereids.trees.expressions.functions.agg; 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.BitmapIntersectFunction; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.util.ExpressionUtils; @@ -30,8 +30,8 @@ /** * AggregateFunction 'orthogonal_bitmap_intersect_count'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapIntersectCount extends AggregateFunction - implements AlwaysNotNullable, OrthogonalBitmapFunction, BitmapIntersectFunction { +public class OrthogonalBitmapIntersectCount extends NotNullableAggregateFunction + implements OrthogonalBitmapFunction, BitmapIntersectFunction { /** * constructor with 3 or more arguments. @@ -44,7 +44,7 @@ public OrthogonalBitmapIntersectCount(Expression arg0, Expression arg1, Expressi * constructor with 3 or more arguments. */ public OrthogonalBitmapIntersectCount(boolean distinct, Expression arg0, Expression arg1, - Expression arg2, Expression... varArgs) { + Expression arg2, Expression... varArgs) { super("orthogonal_bitmap_intersect_count", distinct, ExpressionUtils.mergeArguments(arg0, arg1, arg2, varArgs)); } @@ -63,4 +63,9 @@ public OrthogonalBitmapIntersectCount withDistinctAndChildren(boolean distinct, public R accept(ExpressionVisitor visitor, C context) { return visitor.visitOrthogonalBitmapIntersectCount(this, context); } + + @Override + public Expression resultForEmptyInput() { + return new BigIntLiteral(0); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java index c08bb08aeecf64..081e5d7f6c4250 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/OrthogonalBitmapUnionCount.java @@ -19,8 +19,8 @@ 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.BigIntLiteral; 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; @@ -34,8 +34,8 @@ /** * AggregateFunction 'orthogonal_bitmap_union_count'. This class is generated by GenerateFunction. */ -public class OrthogonalBitmapUnionCount extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class OrthogonalBitmapUnionCount extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE) @@ -73,4 +73,9 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Expression resultForEmptyInput() { + return new BigIntLiteral(0); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java index b4d7467e4c00b7..4412d96006f04d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/PercentileArray.java @@ -19,7 +19,6 @@ 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.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression; @@ -41,8 +40,8 @@ /** * AggregateFunction 'percentile_array'. This class is generated by GenerateFunction. */ -public class PercentileArray extends AggregateFunction - implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class PercentileArray extends NotNullableAggregateFunction + implements BinaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(ArrayType.of(DoubleType.INSTANCE)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java index 3d0729775a5cec..3d107ec282b606 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/QuantileUnion.java @@ -20,7 +20,6 @@ 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.scalar.QuantileStateEmpty; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; @@ -36,8 +35,8 @@ /** * AggregateFunction 'quantile_union'. This class is generated by GenerateFunction. */ -public class QuantileUnion extends AggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable { +public class QuantileUnion extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(QuantileStateType.INSTANCE).args(QuantileStateType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java index 7af112ef8e800f..5ac114f00e83ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/SequenceCount.java @@ -19,7 +19,6 @@ 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.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -39,8 +38,8 @@ /** * AggregateFunction 'sequence_count'. This class is generated by GenerateFunction. */ -public class SequenceCount extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SequenceFunction { +public class SequenceCount extends NotNullableAggregateFunction + implements ExplicitlyCastableSignature, SequenceFunction { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java index fd052a69c0e90e..5a1f0f9fb93d34 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum0.java @@ -20,7 +20,6 @@ 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.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; @@ -53,8 +52,8 @@ * AggregateFunction 'sum0'. sum0 returns the sum of the values which go into it like sum. * It differs in that when no non null values are applied zero is returned instead of null. */ -public class Sum0 extends AggregateFunction - implements UnaryExpression, AlwaysNotNullable, ExplicitlyCastableSignature, ComputePrecisionForSum, +public class Sum0 extends NotNullableAggregateFunction + implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic, RollUpTrait { public static final List SIGNATURES = ImmutableList.of( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java index 802aac3c2233fe..b4591de6af01b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/StringArithmetic.java @@ -151,6 +151,27 @@ private static String trimImpl(String first, String second, boolean left, boolea return result; } + private static String trimInImpl(String first, String second, boolean left, boolean right) { + StringBuilder result = new StringBuilder(first); + + if (left) { + int start = 0; + while (start < result.length() && second.indexOf(result.charAt(start)) != -1) { + start++; + } + result.delete(0, start); + } + if (right) { + int end = result.length(); + while (end > 0 && second.indexOf(result.charAt(end - 1)) != -1) { + end--; + } + result.delete(end, result.length()); + } + + return result.toString(); + } + /** * Executable arithmetic functions Trim */ @@ -199,6 +220,54 @@ public static Expression rtrimVarcharVarchar(StringLikeLiteral first, StringLike return castStringLikeLiteral(first, trimImpl(first.getValue(), second.getValue(), false, true)); } + /** + * Executable arithmetic functions Trim_In + */ + @ExecFunction(name = "trim_in") + public static Expression trimInVarchar(StringLikeLiteral first) { + return castStringLikeLiteral(first, trimInImpl(first.getValue(), " ", true, true)); + } + + /** + * Executable arithmetic functions Trim_In + */ + @ExecFunction(name = "trim_in") + public static Expression trimInVarcharVarchar(StringLikeLiteral first, StringLikeLiteral second) { + return castStringLikeLiteral(first, trimInImpl(first.getValue(), second.getValue(), true, true)); + } + + /** + * Executable arithmetic functions ltrim_in + */ + @ExecFunction(name = "ltrim_in") + public static Expression ltrimInVarchar(StringLikeLiteral first) { + return castStringLikeLiteral(first, trimInImpl(first.getValue(), " ", true, false)); + } + + /** + * Executable arithmetic functions ltrim_in + */ + @ExecFunction(name = "ltrim_in") + public static Expression ltrimInVarcharVarchar(StringLikeLiteral first, StringLikeLiteral second) { + return castStringLikeLiteral(first, trimInImpl(first.getValue(), second.getValue(), true, false)); + } + + /** + * Executable arithmetic functions rtrim_in + */ + @ExecFunction(name = "rtrim_in") + public static Expression rtrimInVarchar(StringLikeLiteral first) { + return castStringLikeLiteral(first, trimInImpl(first.getValue(), " ", false, true)); + } + + /** + * Executable arithmetic functions rtrim_in + */ + @ExecFunction(name = "rtrim_in") + public static Expression rtrimInVarcharVarchar(StringLikeLiteral first, StringLikeLiteral second) { + return castStringLikeLiteral(first, trimInImpl(first.getValue(), second.getValue(), false, true)); + } + /** * Executable arithmetic functions Replace */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/LtrimIn.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/LtrimIn.java new file mode 100644 index 00000000000000..b54723638280e8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/LtrimIn.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.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.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'ltrimIn'. This class is generated by GenerateFunction. + */ +public class LtrimIn extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + private static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) + .args(VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE, StringType.INSTANCE), + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE) + ); + + private LtrimIn(List args) { + super("ltrim_in", args); + } + + /** + * constructor with 1 argument. + */ + public LtrimIn(Expression arg) { + super("ltrim_in", arg); + } + + /** + * constructor with 2 argument. + */ + public LtrimIn(Expression arg0, Expression arg1) { + super("ltrim_in", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public LtrimIn withChildren(List children) { + Preconditions.checkArgument(children.size() == 1 || children.size() == 2); + return new LtrimIn(children); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitLtrimIn(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/RtrimIn.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/RtrimIn.java new file mode 100644 index 00000000000000..035201ef93cb52 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/RtrimIn.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.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.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'rtrim_in'. This class is generated by GenerateFunction. + */ +public class RtrimIn extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + private static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) + .args(VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE, StringType.INSTANCE), + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE) + ); + + private RtrimIn(List args) { + super("rtrim_in", args); + } + + /** + * constructor with 1 argument. + */ + public RtrimIn(Expression arg) { + super("rtrim_in", arg); + } + + /** + * constructor with 2 argument. + */ + public RtrimIn(Expression arg0, Expression arg1) { + super("rtrim_in", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public RtrimIn withChildren(List children) { + Preconditions.checkArgument(children.size() == 1 || children.size() == 2); + return new RtrimIn(children); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitRtrimIn(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TrimIn.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TrimIn.java new file mode 100644 index 00000000000000..978f71e09784b5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/TrimIn.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.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.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * ScalarFunction 'trim_in'. This class is generated by GenerateFunction. + */ +public class TrimIn extends ScalarFunction + implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullable { + + private static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT) + .args(VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE, StringType.INSTANCE), + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(StringType.INSTANCE).args(StringType.INSTANCE) + ); + + private TrimIn(List args) { + super("trim_in", args); + } + + /** + * constructor with 1 argument. + */ + public TrimIn(Expression arg) { + super("trim_in", arg); + } + + /** + * constructor with 2 argument. + */ + public TrimIn(Expression arg0, Expression arg1) { + super("trim_in", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public TrimIn withChildren(List children) { + Preconditions.checkArgument(children.size() == 1 || children.size() == 2); + return new TrimIn(children); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitTrimIn(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 367e91b0abc54d..c63663edd56001 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 @@ -280,6 +280,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Lower; 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.LtrimIn; 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; @@ -355,6 +356,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.RoundBankers; import org.apache.doris.nereids.trees.expressions.functions.scalar.Rpad; import org.apache.doris.nereids.trees.expressions.functions.scalar.Rtrim; +import org.apache.doris.nereids.trees.expressions.functions.scalar.RtrimIn; import org.apache.doris.nereids.trees.expressions.functions.scalar.ScalarFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.Second; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondCeil; @@ -432,6 +434,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize; import org.apache.doris.nereids.trees.expressions.functions.scalar.Translate; import org.apache.doris.nereids.trees.expressions.functions.scalar.Trim; +import org.apache.doris.nereids.trees.expressions.functions.scalar.TrimIn; import org.apache.doris.nereids.trees.expressions.functions.scalar.Truncate; import org.apache.doris.nereids.trees.expressions.functions.scalar.Unhex; import org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp; @@ -1539,6 +1542,10 @@ default R visitLtrim(Ltrim ltrim, C context) { return visitScalarFunction(ltrim, context); } + default R visitLtrimIn(LtrimIn ltrimIn, C context) { + return visitScalarFunction(ltrimIn, context); + } + default R visitMakeDate(MakeDate makeDate, C context) { return visitScalarFunction(makeDate, context); } @@ -1791,6 +1798,10 @@ default R visitRtrim(Rtrim rtrim, C context) { return visitScalarFunction(rtrim, context); } + default R visitRtrimIn(RtrimIn rtrimIn, C context) { + return visitScalarFunction(rtrimIn, context); + } + default R visitSecond(Second second, C context) { return visitScalarFunction(second, context); } @@ -2087,6 +2098,10 @@ default R visitTrim(Trim trim, C context) { return visitScalarFunction(trim, context); } + default R visitTrimIn(TrimIn trimIn, C context) { + return visitScalarFunction(trimIn, context); + } + default R visitTruncate(Truncate truncate, C context) { return visitScalarFunction(truncate, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java index d4fce906693a68..e3f2f1d732ae5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.planner.ScanNode; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; @@ -95,6 +96,9 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } else { executor.handleExplainStmt(planner.getExplainString(explainOptions), true); } + for (ScanNode scanNode : planner.getScanNodes()) { + scanNode.stop(); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index de284bd837748f..36cc0f95a77a8e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -246,11 +246,16 @@ public Plan visitLogicalCTE(LogicalCTE cte, PredicateAddContext if (predicates.isEmpty()) { return cte; } + List> rewrittenSubQueryAlias = new ArrayList<>(); for (LogicalSubQueryAlias subQueryAlias : cte.getAliasQueries()) { + List subQueryAliasChildren = new ArrayList<>(); this.virtualRelationNamePartSet.add(subQueryAlias.getQualifier()); - subQueryAlias.children().forEach(subQuery -> subQuery.accept(this, predicates)); + subQueryAlias.children().forEach(subQuery -> + subQueryAliasChildren.add(subQuery.accept(this, predicates)) + ); + rewrittenSubQueryAlias.add(subQueryAlias.withChildren(subQueryAliasChildren)); } - return super.visitLogicalCTE(cte, predicates); + return super.visitLogicalCTE(new LogicalCTE<>(rewrittenSubQueryAlias, cte.child()), predicates); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 8c44b42a5ccfbd..a3e1f53acd94b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -103,7 +103,8 @@ */ public class CreateMTMVInfo { public static final Logger LOG = LogManager.getLogger(CreateMTMVInfo.class); - public static final String MTMV_PLANER_DISABLE_RULES = "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"; + public static final String MTMV_PLANER_DISABLE_RULES = "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION," + + "ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"; private final boolean ifNotExists; private final TableNameInfo mvName; private List keys; @@ -156,6 +157,9 @@ public void analyze(ConnectContext ctx) throws Exception { if (!InternalCatalog.INTERNAL_CATALOG_NAME.equals(mvName.getCtl())) { throw new AnalysisException("Only support creating asynchronous materialized views in internal catalog"); } + if (ctx.getSessionVariable().isInDebugMode()) { + throw new AnalysisException("Create materialized view fail, because is in debug mode"); + } try { FeNameFormat.checkTableName(mvName.getTbl()); } catch (org.apache.doris.common.AnalysisException e) { @@ -252,9 +256,21 @@ public void analyzeQuery(ConnectContext ctx, Map mvProperties) t NereidsPlanner planner = new NereidsPlanner(statementContext); // this is for expression column name infer when not use alias LogicalSink logicalSink = new UnboundResultSink<>(logicalQuery); - // must disable constant folding by be, because be constant folding may return wrong type - ctx.getSessionVariable().setVarOnce(SessionVariable.ENABLE_FOLD_CONSTANT_BY_BE, "false"); - Plan plan = planner.planWithLock(logicalSink, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + // Should not make table without data to empty relation when analyze the related table, + // so add disable rules + Set tempDisableRules = ctx.getSessionVariable().getDisableNereidsRuleNames(); + ctx.getSessionVariable().setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); + ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + Plan plan; + try { + // must disable constant folding by be, because be constant folding may return wrong type + ctx.getSessionVariable().setVarOnce(SessionVariable.ENABLE_FOLD_CONSTANT_BY_BE, "false"); + plan = planner.planWithLock(logicalSink, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + } finally { + // after operate, roll back the disable rules + ctx.getSessionVariable().setDisableNereidsRules(String.join(",", tempDisableRules)); + ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + } // can not contain VIEW or MTMV analyzeBaseTables(planner.getAnalyzedPlan()); // can not contain Random function @@ -265,8 +281,7 @@ public void analyzeQuery(ConnectContext ctx, Map mvProperties) t throw new AnalysisException("can not contain invalid expression"); } getRelation(planner); - this.mvPartitionInfo = mvPartitionDefinition - .analyzeAndTransferToMTMVPartitionInfo(planner, ctx, logicalQuery); + this.mvPartitionInfo = mvPartitionDefinition.analyzeAndTransferToMTMVPartitionInfo(planner, ctx); this.partitionDesc = generatePartitionDesc(ctx); getColumns(plan, ctx, mvPartitionInfo.getPartitionCol(), distribution); analyzeKeys(); @@ -311,24 +326,10 @@ private void analyzeKeys() { } } + // Should use analyzed plan for collect views and tables private void getRelation(NereidsPlanner planner) { - // Should not make table without data to empty relation when analyze the related table, - // so add disable rules - ConnectContext ctx = planner.getCascadesContext().getConnectContext(); - SessionVariable sessionVariable = ctx.getSessionVariable(); - Set tempDisableRules = sessionVariable.getDisableNereidsRuleNames(); - sessionVariable.setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); - if (ctx.getStatementContext() != null) { - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - } - Plan plan; - try { - plan = planner.planWithLock(logicalQuery, PhysicalProperties.ANY, ExplainLevel.NONE); - } finally { - sessionVariable.setDisableNereidsRules(String.join(",", tempDisableRules)); - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - } - this.relation = MTMVPlanUtil.generateMTMVRelation(plan); + this.relation = MTMVPlanUtil.generateMTMVRelation(planner.getAnalyzedPlan(), + planner.getCascadesContext().getConnectContext()); } private PartitionDesc generatePartitionDesc(ConnectContext ctx) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java index c4117e8608e29d..a26a97f7240793 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java @@ -37,7 +37,6 @@ import org.apache.doris.nereids.analyzer.UnboundFunction; import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.RelatedTableInfo; import org.apache.doris.nereids.trees.expressions.Cast; @@ -45,11 +44,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.SessionVariable; import com.google.common.collect.Sets; @@ -72,11 +67,9 @@ public class MTMVPartitionDefinition { * * @param planner planner * @param ctx ctx - * @param logicalQuery logicalQuery * @return MTMVPartitionInfo */ - public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner planner, ConnectContext ctx, - LogicalPlan logicalQuery) { + public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner planner, ConnectContext ctx) { MTMVPartitionInfo mtmvPartitionInfo = new MTMVPartitionInfo(partitionType); if (this.partitionType == MTMVPartitionType.SELF_MANAGE) { return mtmvPartitionInfo; @@ -100,7 +93,7 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl timeUnit = null; } mtmvPartitionInfo.setPartitionCol(partitionColName); - RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, ctx, logicalQuery, partitionColName, timeUnit); + RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, ctx, partitionColName, timeUnit); mtmvPartitionInfo.setRelatedCol(relatedTableInfo.getColumn()); mtmvPartitionInfo.setRelatedTable(relatedTableInfo.getTableInfo()); if (relatedTableInfo.getPartitionExpression().isPresent()) { @@ -125,47 +118,33 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl return mtmvPartitionInfo; } - private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectContext ctx, LogicalPlan - logicalQuery, - String partitionColName, - String timeUnit) { + // Should use rewritten plan without view and subQuery to get related partition table + private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectContext ctx, + String partitionColName, String timeUnit) { CascadesContext cascadesContext = planner.getCascadesContext(); - SessionVariable sessionVariable = cascadesContext.getConnectContext().getSessionVariable(); - Set tempDisableRules = sessionVariable.getDisableNereidsRuleNames(); - // Should not make table without data to empty relation when analyze the related table, - // so add disable rules - sessionVariable.setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); - cascadesContext.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + + RelatedTableInfo relatedTableInfo = MaterializedViewUtils + .getRelatedTableInfo(partitionColName, timeUnit, planner.getRewrittenPlan(), cascadesContext); + if (!relatedTableInfo.isPctPossible()) { + throw new AnalysisException(String.format("Unable to find a suitable base table for partitioning," + + " the fail reason is %s", relatedTableInfo.getFailReason())); + } + MTMVRelatedTableIf mtmvBaseRealtedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); + Set partitionColumnNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); try { - Plan mvRewrittenPlan = - planner.planWithLock(logicalQuery, PhysicalProperties.ANY, ExplainLevel.REWRITTEN_PLAN); - RelatedTableInfo relatedTableInfo = MaterializedViewUtils - .getRelatedTableInfo(partitionColName, timeUnit, mvRewrittenPlan, cascadesContext); - if (!relatedTableInfo.isPctPossible()) { - throw new AnalysisException(String.format("Unable to find a suitable base table for partitioning," - + " the fail reason is %s", relatedTableInfo.getFailReason())); - } - MTMVRelatedTableIf mtmvBaseRealtedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); - Set partitionColumnNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - try { - partitionColumnNames.addAll(mtmvBaseRealtedTable.getPartitionColumnNames(Optional.empty())); - } catch (DdlException e) { - throw new AnalysisException(e.getMessage(), e); - } + partitionColumnNames.addAll(mtmvBaseRealtedTable.getPartitionColumnNames(Optional.empty())); + } catch (DdlException e) { + throw new AnalysisException(e.getMessage(), e); + } - if (!partitionColumnNames.contains(relatedTableInfo.getColumn())) { - throw new AnalysisException("error related column: " + relatedTableInfo.getColumn()); - } - if (!(mtmvBaseRealtedTable instanceof HMSExternalTable) - && partitionColumnNames.size() != 1) { - throw new AnalysisException("only hms table support multi column partition."); - } - return relatedTableInfo; - } finally { - // after operate, roll back the disable rules - sessionVariable.setDisableNereidsRules(String.join(",", tempDisableRules)); - cascadesContext.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + if (!partitionColumnNames.contains(relatedTableInfo.getColumn())) { + throw new AnalysisException("error related column: " + relatedTableInfo.getColumn()); + } + if (!(mtmvBaseRealtedTable instanceof HMSExternalTable) + && partitionColumnNames.size() != 1) { + throw new AnalysisException("only hms table support multi column partition."); } + return relatedTableInfo; } private static List convertToLegacyArguments(List children) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ShowCreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ShowCreateMTMVInfo.java index 588af2365741c5..b4fde11c3e1e64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ShowCreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ShowCreateMTMVInfo.java @@ -74,18 +74,13 @@ public void analyze(ConnectContext ctx) { * @throws DdlException DdlException * @throws IOException IOException */ - public void run(StmtExecutor executor) throws DdlException, IOException { + public void run(StmtExecutor executor) throws DdlException, IOException, org.apache.doris.common.AnalysisException { List> rows = Lists.newArrayList(); Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(mvName.getDb()); MTMV mtmv = (MTMV) db.getTableOrDdlException(mvName.getTbl()); - mtmv.readLock(); - try { - String mtmvDdl = Env.getMTMVDdl(mtmv); - rows.add(Lists.newArrayList(mtmv.getName(), mtmvDdl)); - executor.handleShowCreateMTMVStmt(rows); - } finally { - mtmv.readUnlock(); - } + String mtmvDdl = Env.getMTMVDdl(mtmv); + rows.add(Lists.newArrayList(mtmv.getName(), mtmvDdl)); + executor.handleShowCreateMTMVStmt(rows); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java index 6b301c16f73aa2..60e7e5bf805a64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java @@ -297,6 +297,7 @@ public static Plan normalizePlan(Plan plan, TableIf table, Optional insertCol = unboundLogicalSink.getColNames().stream() .filter(c -> c.equalsIgnoreCase(col.getName())).findFirst(); @@ -313,9 +314,18 @@ public static Plan normalizePlan(Plan plan, TableIf table, Optional) unboundLogicalSink).setPartialUpdate(false); + } else { + if (hasMissingAutoIncKey) { + // becuase of the uniqueness of genetaed value of auto-increment column, + // we convert this load to upsert when is misses auto-increment key column + ((UnboundTableSink) unboundLogicalSink).setPartialUpdate(false); + } } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java index 9f65988da95358..31cee19cc43492 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java @@ -24,6 +24,8 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregatePhase; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; import org.apache.doris.nereids.trees.expressions.functions.agg.Ndv; import org.apache.doris.nereids.trees.plans.Plan; @@ -267,6 +269,12 @@ public LogicalAggregate withGroupByAndOutput(List groupByExprL hasPushed, sourceRepeat, Optional.empty(), Optional.empty(), child()); } + public LogicalAggregate withChildGroupByAndOutput(List groupByExprList, + List outputExpressionList, Plan newChild) { + return new LogicalAggregate<>(groupByExprList, outputExpressionList, normalized, ordinalIsResolved, generated, + hasPushed, sourceRepeat, Optional.empty(), Optional.empty(), newChild); + } + public LogicalAggregate withChildAndOutput(CHILD_TYPE child, List outputExpressionList) { return new LogicalAggregate<>(groupByExpressions, outputExpressionList, normalized, ordinalIsResolved, @@ -382,4 +390,14 @@ public void computeEqualSet(DataTrait.Builder builder) { public void computeFd(DataTrait.Builder builder) { builder.addFuncDepsDG(child().getLogicalProperties().getTrait()); } + + /** supportAggregatePhase */ + public boolean supportAggregatePhase(AggregatePhase aggregatePhase) { + for (AggregateFunction aggregateFunction : getAggregateFunctions()) { + if (!aggregateFunction.supportAggregatePhase(aggregatePhase)) { + return false; + } + } + return true; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java index d23ea3d2395f05..efd7e90c13615e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java @@ -37,6 +37,7 @@ import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -154,9 +155,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java index f4f2178840b6ab..680988b39f6bb1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHaving.java @@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -125,9 +126,9 @@ public void computeUnique(Builder builder) { @Override public void computeUniform(Builder builder) { for (Expression e : getConjuncts()) { - Set uniformSlots = ExpressionUtils.extractUniformSlot(e); - for (Slot slot : uniformSlots) { - builder.addUniformSlot(slot); + Map uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Map.Entry entry : uniformSlots.entrySet()) { + builder.addUniformSlotAndLiteral(entry.getKey(), entry.getValue()); } } builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHudiScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHudiScan.java index 629690889432b3..51e68eb07631ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHudiScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalHudiScan.java @@ -84,7 +84,7 @@ protected LogicalHudiScan(RelationId id, ExternalTable table, List quali public LogicalHudiScan(RelationId id, ExternalTable table, List qualifier, Optional tableSample, Optional tableSnapshot) { this(id, table, qualifier, Optional.empty(), Optional.empty(), - SelectedPartitions.NOT_PRUNED, tableSample, tableSnapshot, + ((HMSExternalTable) table).initHudiSelectedPartitions(tableSnapshot), tableSample, tableSnapshot, Optional.empty(), Optional.empty()); } 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 f557b07d3b646e..c583360c3d8a76 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 @@ -535,11 +535,35 @@ public void computeUniform(Builder builder) { // TODO disable function dependence calculation for mark join, but need re-think this in future. return; } - if (!joinType.isLeftSemiOrAntiJoin()) { - builder.addUniformSlot(right().getLogicalProperties().getTrait()); - } - if (!joinType.isRightSemiOrAntiJoin()) { - builder.addUniformSlot(left().getLogicalProperties().getTrait()); + switch (joinType) { + case INNER_JOIN: + case CROSS_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + break; + case LEFT_SEMI_JOIN: + case LEFT_ANTI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + break; + case RIGHT_SEMI_JOIN: + case RIGHT_ANTI_JOIN: + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + break; + case LEFT_OUTER_JOIN: + builder.addUniformSlot(left().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(right().getLogicalProperties().getTrait()); + break; + case RIGHT_OUTER_JOIN: + builder.addUniformSlot(right().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(left().getLogicalProperties().getTrait()); + break; + case FULL_OUTER_JOIN: + builder.addUniformSlotForOuterJoinNullableSide(left().getLogicalProperties().getTrait()); + builder.addUniformSlotForOuterJoinNullableSide(right().getLogicalProperties().getTrait()); + break; + default: + break; } } 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 90ceb24231340c..2216e58c4fa3b8 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 @@ -99,6 +99,12 @@ public class LogicalOlapScan extends LogicalCatalogRelation implements OlapScan */ private final List selectedTabletIds; + /** + * Selected tablet ids to read data from, this would be set if user query with tablets manually + * Such as select * from orders TABLET(100); + */ + private final List manuallySpecifiedTabletIds; + /////////////////////////////////////////////////////////////////////////// // Members for partition ids. /////////////////////////////////////////////////////////////////////////// @@ -127,12 +133,16 @@ public LogicalOlapScan(RelationId id, OlapTable table) { this(id, table, ImmutableList.of()); } + /** + * LogicalOlapScan construct method + */ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier) { this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, ImmutableList.of(), -1, false, PreAggStatus.unset(), ImmutableList.of(), ImmutableList.of(), - Maps.newHashMap(), Optional.empty(), false, ImmutableMap.of()); + Maps.newHashMap(), Optional.empty(), false, ImmutableMap.of(), + ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List tabletIds, @@ -140,7 +150,7 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, tabletIds, -1, false, PreAggStatus.unset(), ImmutableList.of(), hints, Maps.newHashMap(), - tableSample, false, ImmutableMap.of()); + tableSample, false, ImmutableMap.of(), ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List specifiedPartitions, @@ -149,7 +159,7 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L // must use specifiedPartitions here for prune partition by sql like 'select * from t partition p1' specifiedPartitions, false, tabletIds, -1, false, PreAggStatus.unset(), specifiedPartitions, hints, Maps.newHashMap(), - tableSample, false, ImmutableMap.of()); + tableSample, false, ImmutableMap.of(), ImmutableList.of()); } public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List tabletIds, @@ -158,7 +168,8 @@ public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, L this(id, table, qualifier, Optional.empty(), Optional.empty(), selectedPartitionIds, false, tabletIds, selectedIndexId, true, preAggStatus, - specifiedPartitions, hints, Maps.newHashMap(), tableSample, true, ImmutableMap.of()); + specifiedPartitions, hints, Maps.newHashMap(), tableSample, true, ImmutableMap.of(), + ImmutableList.of()); } /** @@ -171,7 +182,7 @@ public LogicalOlapScan(RelationId id, Table table, List qualifier, PreAggStatus preAggStatus, List specifiedPartitions, List hints, Map, Slot> cacheSlotWithSlotName, Optional tableSample, boolean directMvScan, - Map>> colToSubPathsMap) { + Map>> colToSubPathsMap, List specifiedTabletIds) { super(id, PlanType.LOGICAL_OLAP_SCAN, table, qualifier, groupExpression, logicalProperties); Preconditions.checkArgument(selectedPartitionIds != null, @@ -182,6 +193,7 @@ public LogicalOlapScan(RelationId id, Table table, List qualifier, this.indexSelected = indexSelected; this.preAggStatus = preAggStatus; this.manuallySpecifiedPartitions = ImmutableList.copyOf(specifiedPartitions); + this.manuallySpecifiedTabletIds = ImmutableList.copyOf(specifiedTabletIds); if (selectedPartitionIds.isEmpty()) { this.selectedPartitionIds = ImmutableList.of(); @@ -240,6 +252,7 @@ public boolean equals(Object o) { && partitionPruned == that.partitionPruned && Objects.equals(preAggStatus, that.preAggStatus) && Objects.equals(selectedTabletIds, that.selectedTabletIds) && Objects.equals(manuallySpecifiedPartitions, that.manuallySpecifiedPartitions) + && Objects.equals(manuallySpecifiedTabletIds, that.manuallySpecifiedTabletIds) && Objects.equals(selectedPartitionIds, that.selectedPartitionIds) && Objects.equals(hints, that.hints) && Objects.equals(tableSample, that.tableSample); @@ -248,8 +261,8 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(super.hashCode(), selectedIndexId, indexSelected, preAggStatus, cacheSlotWithSlotName, - selectedTabletIds, partitionPruned, manuallySpecifiedPartitions, selectedPartitionIds, hints, - tableSample); + selectedTabletIds, partitionPruned, manuallySpecifiedTabletIds, manuallySpecifiedPartitions, + selectedPartitionIds, hints, tableSample); } @Override @@ -258,7 +271,7 @@ public LogicalOlapScan withGroupExpression(Optional groupExpres groupExpression, Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } @Override @@ -267,7 +280,7 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr return new LogicalOlapScan(relationId, (Table) table, qualifier, groupExpression, logicalProperties, selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) { @@ -275,7 +288,7 @@ public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, true, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withMaterializedIndexSelected(long indexId) { @@ -283,7 +296,7 @@ public LogicalOlapScan withMaterializedIndexSelected(long indexId) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, indexId, true, PreAggStatus.unset(), manuallySpecifiedPartitions, hints, cacheSlotWithSlotName, - tableSample, directMvScan, colToSubPathsMap); + tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { @@ -291,7 +304,7 @@ public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { @@ -299,7 +312,7 @@ public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } public LogicalOlapScan withColToSubPathsMap(Map>> colToSubPathsMap) { @@ -307,7 +320,15 @@ public LogicalOlapScan withColToSubPathsMap(Map>> colTo Optional.empty(), Optional.empty(), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap); + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); + } + + public LogicalOlapScan withManuallySpecifiedTabletIds(List manuallySpecifiedTabletIds) { + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), + selectedPartitionIds, partitionPruned, selectedTabletIds, + selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, + hints, cacheSlotWithSlotName, tableSample, directMvScan, colToSubPathsMap, manuallySpecifiedTabletIds); } @Override @@ -317,7 +338,7 @@ public LogicalOlapScan withRelationId(RelationId relationId) { Optional.empty(), Optional.empty(), selectedPartitionIds, false, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, - hints, Maps.newHashMap(), tableSample, directMvScan, colToSubPathsMap); + hints, Maps.newHashMap(), tableSample, directMvScan, colToSubPathsMap, selectedTabletIds); } @Override @@ -333,6 +354,10 @@ public List getSelectedTabletIds() { return selectedTabletIds; } + public List getManuallySpecifiedTabletIds() { + return manuallySpecifiedTabletIds; + } + @Override public long getSelectedIndexId() { return selectedIndexId; 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 58d99b635e78ec..159552b4653b07 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 @@ -248,14 +248,18 @@ public void computeUnique(DataTrait.Builder builder) { public void computeUniform(DataTrait.Builder builder) { builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); for (NamedExpression proj : getProjects()) { - if (proj.children().isEmpty()) { + if (!(proj instanceof Alias)) { continue; } if (proj.child(0).isConstant()) { - builder.addUniformSlot(proj.toSlot()); - } else if (ExpressionUtils.isInjective(proj.child(0))) { - ImmutableSet inputs = ImmutableSet.copyOf(proj.getInputSlots()); - if (child(0).getLogicalProperties().getTrait().isUniform(inputs)) { + builder.addUniformSlotAndLiteral(proj.toSlot(), proj.child(0)); + } else if (proj.child(0) instanceof Slot) { + Slot slot = (Slot) proj.child(0); + DataTrait childTrait = child(0).getLogicalProperties().getTrait(); + if (childTrait.isUniformAndHasConstValue(slot)) { + builder.addUniformSlotAndLiteral(proj.toSlot(), + child(0).getLogicalProperties().getTrait().getUniformValue(slot).get()); + } else if (childTrait.isUniform(slot)) { builder.addUniformSlot(proj.toSlot()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java index 2e2cdb810f0f72..27ff1e4b68c075 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java @@ -20,8 +20,8 @@ import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.common.AnalysisException; import org.apache.doris.mtmv.MTMVCache; -import org.apache.doris.mtmv.MTMVPlanUtil; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; @@ -70,13 +70,19 @@ public Plan visitPhysicalCatalogRelation(PhysicalCatalogRelation catalogRelation } private void expandMvAndCollect(MTMV mtmv, TableCollectorContext context) { - if (!context.isExpand()) { + if (!context.isExpandMaterializedView()) { return; } // Make sure use only one connection context when in query to avoid ConnectionContext.get() wrong - MTMVCache expandedMv = MTMVCache.from(mtmv, context.getConnectContext() == null - ? MTMVPlanUtil.createMTMVContext(mtmv) : context.getConnectContext(), false); - expandedMv.getLogicalPlan().accept(this, context); + MTMVCache expandedMvCache; + try { + expandedMvCache = mtmv.getOrGenerateCache(context.getConnectContext()); + } catch (AnalysisException exception) { + LOG.warn(String.format("expandMvAndCollect getOrGenerateCache fail, mtmv name is %s", mtmv.getName()), + exception); + expandedMvCache = MTMVCache.from(mtmv, context.getConnectContext(), false); + } + expandedMvCache.getAnalyzedPlan().accept(this, context); } /** @@ -87,12 +93,14 @@ public static final class TableCollectorContext { private final Set collectedTables = new HashSet<>(); private final Set targetTableTypes; // if expand the mv or not - private final boolean expand; - private ConnectContext connectContext; + private final boolean expandMaterializedView; + private final ConnectContext connectContext; - public TableCollectorContext(Set targetTableTypes, boolean expand) { + public TableCollectorContext(Set targetTableTypes, boolean expandMaterializedView, + ConnectContext connectContext) { this.targetTableTypes = targetTableTypes; - this.expand = expand; + this.expandMaterializedView = expandMaterializedView; + this.connectContext = connectContext; } public Set getCollectedTables() { @@ -103,16 +111,12 @@ public Set getTargetTableTypes() { return targetTableTypes; } - public boolean isExpand() { - return expand; + public boolean isExpandMaterializedView() { + return expandMaterializedView; } public ConnectContext getConnectContext() { return connectContext; } - - public void setConnectContext(ConnectContext connectContext) { - this.connectContext = connectContext; - } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 67734f66aa17c1..b303fd2a6b59fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -63,6 +63,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -726,15 +727,15 @@ public static List collectToList(Collection express /** * extract uniform slot for the given predicate, such as a = 1 and b = 2 */ - public static ImmutableSet extractUniformSlot(Expression expression) { - ImmutableSet.Builder builder = new ImmutableSet.Builder<>(); + public static ImmutableMap extractUniformSlot(Expression expression) { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); if (expression instanceof And) { - builder.addAll(extractUniformSlot(expression.child(0))); - builder.addAll(extractUniformSlot(expression.child(1))); + builder.putAll(extractUniformSlot(expression.child(0))); + builder.putAll(extractUniformSlot(expression.child(1))); } if (expression instanceof EqualTo) { if (isInjective(expression.child(0)) && expression.child(1).isConstant()) { - builder.add((Slot) expression.child(0)); + builder.put((Slot) expression.child(0), expression.child(1)); } } return builder.build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java index 260ad316d3cc24..8e4e9b9af8e0d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/BatchDropInfo.java @@ -86,6 +86,10 @@ public Set getIndexIdSet() { return indexIdSet; } + public boolean hasIndexNameMap() { + return indexNameMap != null; + } + public Map getIndexNameMap() { return indexNameMap; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java index 69994caf23d5dc..db0688bd6ad267 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java @@ -52,7 +52,7 @@ public DropInfo() { public DropInfo(long dbId, long tableId, String tableName, boolean isView, boolean forceDrop, long recycleTime) { - this(dbId, tableId, tableName, -1, "", isView, forceDrop, recycleTime); + this(dbId, tableId, tableName, -1L, "", isView, forceDrop, recycleTime); } public DropInfo(long dbId, long tableId, String tableName, long indexId, String indexName, boolean isView, 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 7d1f2127eecaaf..f1377e9daebfc4 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 @@ -347,13 +347,21 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { } case OperationType.OP_BATCH_DROP_ROLLUP: { BatchDropInfo batchDropInfo = (BatchDropInfo) journal.getData(); - for (Map.Entry entry : batchDropInfo.getIndexNameMap().entrySet()) { - long indexId = entry.getKey(); - String indexName = entry.getValue(); - DropInfo info = new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), - batchDropInfo.getTableName(), indexId, indexName, false, false, 0); - env.getMaterializedViewHandler().replayDropRollup(info, env); - env.getBinlogManager().addDropRollup(info, logId); + if (batchDropInfo.hasIndexNameMap()) { + for (Map.Entry entry : batchDropInfo.getIndexNameMap().entrySet()) { + long indexId = entry.getKey(); + String indexName = entry.getValue(); + DropInfo info = new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), + batchDropInfo.getTableName(), indexId, indexName, false, false, 0); + env.getMaterializedViewHandler().replayDropRollup(info, env); + env.getBinlogManager().addDropRollup(info, logId); + } + } else { + for (Long indexId : batchDropInfo.getIndexIdSet()) { + DropInfo info = new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), + batchDropInfo.getTableName(), indexId, "", false, false, 0); + env.getMaterializedViewHandler().replayDropRollup(info, env); + } } break; } 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 c8ef2e22662b4a..f78b0735a1e83f 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 @@ -79,10 +79,6 @@ public OriginalPlanner(Analyzer analyzer) { this.analyzer = analyzer; } - public boolean isBlockQuery() { - return isBlockQuery; - } - public PlannerContext getPlannerContext() { return plannerContext; } @@ -276,17 +272,6 @@ public void createPlanFragments(StatementBase statement, Analyzer analyzer, TQue if (queryStmt instanceof SelectStmt) { SelectStmt selectStmt = (SelectStmt) queryStmt; - if (queryStmt.getSortInfo() != null || selectStmt.getAggInfo() != null) { - isBlockQuery = true; - if (LOG.isDebugEnabled()) { - LOG.debug("this is block query"); - } - } else { - isBlockQuery = false; - if (LOG.isDebugEnabled()) { - LOG.debug("this isn't block query"); - } - } if (selectStmt.isTwoPhaseReadOptEnabled()) { // Optimize query like `SELECT ... FROM WHERE ... ORDER BY ... LIMIT ...` if (singleNodePlan instanceof SortNode 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 5617ad57e8f66e..cfcd27af8fa5d2 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 @@ -44,8 +44,6 @@ public abstract class Planner { protected ArrayList fragments = Lists.newArrayList(); - protected boolean isBlockQuery = false; - protected TQueryOptions queryOptions; public abstract List getScanNodes(); @@ -116,10 +114,6 @@ public List getFragments() { return fragments; } - public boolean isBlockQuery() { - return isBlockQuery; - } - public TQueryOptions getQueryOptions() { return queryOptions; } 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 d94ad0a2552240..ae5d562bd5bd30 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 @@ -76,7 +76,6 @@ import org.apache.doris.datasource.trinoconnector.source.TrinoConnectorScanNode; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException; -import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TPushAggOp; import com.google.common.base.Preconditions; @@ -1969,7 +1968,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s + "please set enable_nereids_planner = true to enable new optimizer"); } scanNode = new HudiScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true, - Optional.empty(), Optional.empty()); + Optional.empty(), Optional.empty(), ConnectContext.get().getSessionVariable()); break; case ICEBERG: scanNode = new IcebergScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); @@ -1993,9 +1992,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s scanNode = new TrinoConnectorScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case MAX_COMPUTE_EXTERNAL_TABLE: - // TODO: support max compute scan node - scanNode = new MaxComputeScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "MCScanNode", - StatisticalType.MAX_COMPUTE_SCAN_NODE, true); + scanNode = new MaxComputeScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case ES_EXTERNAL_TABLE: scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java index 0aecce4360fe1a..7f51294054d22a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java @@ -272,7 +272,7 @@ private static void logAuditLogImpl(ConnectContext ctx, String origStmt, Stateme auditEventBuilder.setStmtType(getStmtType(parsedStmt)); if (!Env.getCurrentEnv().isMaster()) { - if (ctx.executor.isForwardToMaster()) { + if (ctx.executor != null && ctx.executor.isForwardToMaster()) { auditEventBuilder.setState(ctx.executor.getProxyStatus()); int proxyStatusCode = ctx.executor.getProxyStatusCode(); if (proxyStatusCode != 0) { 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 78493a46ad192a..0f6bc0212d1f89 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 @@ -234,8 +234,6 @@ public class Coordinator implements CoordInterface { // same as backend_exec_states_.size() after Exec() private final Set instanceIds = Sets.newHashSet(); - private final boolean isBlockQuery; - private int numReceivedRows = 0; private List deltaUrls; @@ -331,7 +329,6 @@ public Coordinator(ConnectContext context, Analyzer analyzer, Planner planner, // Used for query/insert/test public Coordinator(ConnectContext context, Analyzer analyzer, Planner planner) { this.context = context; - this.isBlockQuery = planner.isBlockQuery(); this.queryId = context.queryId(); this.fragments = planner.getFragments(); this.scanNodes = planner.getScanNodes(); @@ -374,7 +371,6 @@ public Coordinator(ConnectContext context, Analyzer analyzer, Planner planner) { // Constructor of Coordinator is too complicated. public Coordinator(Long jobId, TUniqueId queryId, DescriptorTable descTable, List fragments, List scanNodes, String timezone, boolean loadZeroTolerance, boolean enableProfile) { - this.isBlockQuery = true; this.jobId = jobId; this.queryId = queryId; this.descTable = descTable.toThrift(); @@ -1206,23 +1202,22 @@ public RowBatch getNext() throws Exception { numReceivedRows += resultBatch.getBatch().getRowsSize(); } + // if reached limit rows, cancel this query immediately + // to avoid BE from reading more data. + // ATTN: if change here, also need to change the same logic in QueryProcessor.getNext(); + Long limitRows = fragments.get(0).getPlanRoot().getLimit(); + boolean reachedLimit = LimitUtils.cancelIfReachLimit( + resultBatch, limitRows, numReceivedRows, this::cancelInternal); + if (resultBatch.isEos()) { receivers.remove(receiver); if (receivers.isEmpty()) { returnedAllResults = true; - } else { + } else if (!reachedLimit) { + // if reachedLimit is true, which means this query has been cancelled. + // so no need to set eos to false again. resultBatch.setEos(false); } - - // if this query is a block query do not cancel. - Long numLimitRows = fragments.get(0).getPlanRoot().getLimit(); - boolean hasLimit = numLimitRows > 0; - if (!isBlockQuery && instanceIds.size() > 1 && hasLimit && numReceivedRows >= numLimitRows) { - if (LOG.isDebugEnabled()) { - LOG.debug("no block query, return num >= limit rows, need cancel"); - } - cancelInternal(new Status(TStatusCode.LIMIT_REACH, "query reach limit")); - } } if (!returnedAllResults) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/LimitUtils.java b/fe/fe-core/src/main/java/org/apache/doris/qe/LimitUtils.java new file mode 100644 index 00000000000000..cbbe5c71a0ff06 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/LimitUtils.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.qe; + +import org.apache.doris.common.Status; +import org.apache.doris.thrift.TStatusCode; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.function.Consumer; + +/** + * This is a utility class for limit related operations. + * Because current there are 2 places need to check limit rows, so put the logic here for unification. + * - Coordinator.getNext(); + * - QueryProcessor.getNext(); + */ +public class LimitUtils { + private static final Logger LOG = LogManager.getLogger(LimitUtils.class); + private static final Status LIMIT_REACH_STATUS = new Status(TStatusCode.LIMIT_REACH, "query reach limit"); + + // if reached limit rows, cancel this query immediately + // to avoid BE from reading more data. + public static boolean cancelIfReachLimit(RowBatch resultBatch, long limitRows, long numReceivedRows, + Consumer cancelFunc) { + boolean reachedLimit = false; + if (limitRows > 0 && numReceivedRows >= limitRows) { + if (LOG.isDebugEnabled()) { + LOG.debug("reach limit rows: {}, received rows: {}, cancel query", limitRows, numReceivedRows); + } + cancelFunc.accept(LIMIT_REACH_STATUS); + // set this + resultBatch.setEos(true); + reachedLimit = true; + } + return reachedLimit; + } +} 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 3ac50c9d7b53a1..675bcbaf7988d5 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 @@ -633,6 +633,8 @@ public class SessionVariable implements Serializable, Writable { public static final String FORCE_JNI_SCANNER = "force_jni_scanner"; + public static final String HUDI_JNI_SCANNER = "hudi_jni_scanner"; + public static final String ENABLE_COUNT_PUSH_DOWN_FOR_EXTERNAL_TABLE = "enable_count_push_down_for_external_table"; public static final String SHOW_ALL_FE_CONNECTION = "show_all_fe_connection"; @@ -721,7 +723,7 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = ENABLE_JDBC_CAST_PREDICATE_PUSH_DOWN, needForward = true, description = {"是否允许将带有 CAST 表达式的谓词下推到 JDBC 外部表。", "Whether to allow predicates with CAST expressions to be pushed down to JDBC external tables."}) - public boolean enableJdbcCastPredicatePushDown = false; + public boolean enableJdbcCastPredicatePushDown = true; @VariableMgr.VarAttr(name = ROUND_PRECISE_DECIMALV2_VALUE) public boolean roundPreciseDecimalV2Value = false; @@ -2072,6 +2074,10 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { description = {"强制使用jni方式读取外表", "Force the use of jni mode to read external table"}) private boolean forceJniScanner = false; + @VariableMgr.VarAttr(name = HUDI_JNI_SCANNER, description = { "使用那种hudi jni scanner, 'hadoop' 或 'spark'", + "Which hudi jni scanner to use, 'hadoop' or 'spark'" }) + private String hudiJniScanner = "hadoop"; + @VariableMgr.VarAttr(name = ENABLE_COUNT_PUSH_DOWN_FOR_EXTERNAL_TABLE, description = {"对外表启用 count(*) 下推优化", "enable count(*) pushdown optimization for external table"}) private boolean enableCountPushDownForExternalTable = true; @@ -4482,6 +4488,10 @@ public boolean isForceJniScanner() { return forceJniScanner; } + public String getHudiJniScanner() { + return hudiJniScanner; + } + public String getIgnoreSplitType() { return ignoreSplitType; } @@ -4502,6 +4512,10 @@ public void setForceJniScanner(boolean force) { forceJniScanner = force; } + public void setHudiJniScanner(String hudiJniScanner) { + this.hudiJniScanner = hudiJniScanner; + } + public boolean isEnableCountPushDownForExternalTable() { return enableCountPushDownForExternalTable; } 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 9bda42139ef639..3dcb7069abf423 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 @@ -1184,15 +1184,9 @@ private void handleShowCreateMTMV() throws AnalysisException { .getDbOrAnalysisException(showStmt.getDb()); MTMV mtmv = (MTMV) db.getTableOrAnalysisException(showStmt.getTable()); List> rows = Lists.newArrayList(); - - mtmv.readLock(); - try { - String mtmvDdl = Env.getMTMVDdl(mtmv); - rows.add(Lists.newArrayList(mtmv.getName(), mtmvDdl)); - resultSet = new ShowResultSet(showStmt.getMetaData(), rows); - } finally { - mtmv.readUnlock(); - } + String mtmvDdl = Env.getMTMVDdl(mtmv); + rows.add(Lists.newArrayList(mtmv.getName(), mtmvDdl)); + resultSet = new ShowResultSet(showStmt.getMetaData(), rows); } // Describe statement 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 345b31d6bc2537..8144ca22ea22eb 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 @@ -20,6 +20,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.CustomThreadFactory; import org.apache.doris.scheduler.constants.TaskType; +import org.apache.doris.scheduler.exception.JobException; import com.lmax.disruptor.EventTranslatorThreeArg; import com.lmax.disruptor.LiteTimeoutBlockingWaitStrategy; @@ -119,15 +120,17 @@ public void tryPublish(Long jobId, Long taskId, TaskType taskType) { * * @param taskId task id */ - public void tryPublishTask(Long taskId) { + public void tryPublishTask(Long taskId) throws JobException { if (isClosed) { log.info("tryPublish failed, disruptor is closed, taskId: {}", taskId); return; } - try { + // We reserve two slots in the ring buffer + // to prevent it from becoming stuck due to competition between producers and consumers. + if (disruptor.getRingBuffer().hasAvailableCapacity(2)) { disruptor.publishEvent(TRANSLATOR, taskId, 0L, TaskType.TRANSIENT_TASK); - } catch (Exception e) { - log.warn("tryPublish failed, taskId: {}", taskId, e); + } else { + throw new JobException("There is not enough available capacity in the RingBuffer."); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java index 7461399c8eb0c5..de501d3e0c2ffd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java @@ -21,7 +21,6 @@ import org.apache.doris.scheduler.exception.JobException; import org.apache.doris.scheduler.executor.TransientTaskExecutor; -import lombok.Setter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -41,7 +40,6 @@ public class TransientTaskManager { * disruptor is used to handle task * disruptor will start a thread pool to handle task */ - @Setter private TaskDisruptor disruptor; public TransientTaskManager() { @@ -56,7 +54,7 @@ public TransientTaskExecutor getMemoryTaskExecutor(Long taskId) { return taskExecutorMap.get(taskId); } - public Long addMemoryTask(TransientTaskExecutor executor) { + public Long addMemoryTask(TransientTaskExecutor executor) throws JobException { Long taskId = executor.getId(); taskExecutorMap.put(taskId, executor); disruptor.tryPublishTask(taskId); 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 cf9e8e82ce112f..1ad8d733ddea07 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 @@ -972,6 +972,10 @@ private TColumnDesc getColumnDesc(Column column) { } desc.setChildren(children); } + String defaultValue = column.getDefaultValue(); + if (defaultValue != null) { + desc.setDefaultValue(defaultValue); + } return desc; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java index febadbef0ab0d2..9ba7a1118e674f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/arrowflight/FlightSqlConnectProcessor.java @@ -186,11 +186,11 @@ public Schema fetchArrowFlightSchema(int timeoutMs) { public void close() throws Exception { ctx.setCommand(MysqlCommand.COM_SLEEP); ctx.clear(); - // TODO support query profile for (StmtExecutor asynExecutor : returnResultFromRemoteExecutor) { asynExecutor.finalizeQuery(); } returnResultFromRemoteExecutor.clear(); + executor.finalizeQuery(); ConnectContext.remove(); } } 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 6ec413821ea37b..58b2c3e3d1f8dc 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 @@ -183,6 +183,9 @@ public enum ScheduleType { @SerializedName("updateRows") public final long updateRows; + @SerializedName("tv") + public final long tableVersion; + public final Map partitionUpdateRows = new ConcurrentHashMap<>(); @SerializedName("tblUpdateTime") @@ -206,8 +209,8 @@ public AnalysisInfo(long jobId, long taskId, List taskIds, long catalogId, long lastExecTimeInMs, long timeCostInMs, AnalysisState state, ScheduleType scheduleType, boolean partitionOnly, boolean samplingPartition, boolean isAllPartition, long partitionCount, CronExpression cronExpression, boolean forceFull, - boolean usingSqlForExternalTable, long tblUpdateTime, long rowCount, boolean userInject, - long updateRows, JobPriority priority, Map partitionUpdateRows, boolean enablePartition) { + boolean usingSqlForExternalTable, long tblUpdateTime, long rowCount, boolean userInject, long updateRows, + long tableVersion, JobPriority priority, Map partitionUpdateRows, boolean enablePartition) { this.jobId = jobId; this.taskId = taskId; this.taskIds = taskIds; @@ -244,6 +247,7 @@ public AnalysisInfo(long jobId, long taskId, List taskIds, long catalogId, this.rowCount = rowCount; this.userInject = userInject; this.updateRows = updateRows; + this.tableVersion = tableVersion; this.priority = priority; if (partitionUpdateRows != null) { this.partitionUpdateRows.putAll(partitionUpdateRows); @@ -292,6 +296,7 @@ public String toString() { sj.add("rowCount: " + rowCount); sj.add("userInject: " + userInject); sj.add("updateRows: " + updateRows); + sj.add("tableVersion: " + tableVersion); sj.add("priority: " + priority.name()); sj.add("enablePartition: " + enablePartition); return sj.toString(); 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 73817363ef164f..bbd0d6164954be 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 @@ -63,6 +63,7 @@ public class AnalysisInfoBuilder { private long rowCount; private boolean userInject = false; private long updateRows; + private long tableVersion; private JobPriority priority; private Map partitionUpdateRows; private boolean enablePartition; @@ -104,6 +105,7 @@ public AnalysisInfoBuilder(AnalysisInfo info) { rowCount = info.rowCount; userInject = info.userInject; updateRows = info.updateRows; + tableVersion = info.tableVersion; priority = info.priority; partitionUpdateRows = info.partitionUpdateRows; enablePartition = info.enablePartition; @@ -274,6 +276,11 @@ public AnalysisInfoBuilder setUpdateRows(long updateRows) { return this; } + public AnalysisInfoBuilder setTableVersion(long tableVersion) { + this.tableVersion = tableVersion; + return this; + } + public AnalysisInfoBuilder setPriority(JobPriority priority) { this.priority = priority; return this; @@ -295,7 +302,7 @@ public AnalysisInfo build() { sampleRows, maxBucketNum, periodTimeInMs, message, lastExecTimeInMs, timeCostInMs, state, scheduleType, partitionOnly, samplingPartition, isAllPartition, partitionCount, cronExpression, forceFull, usingSqlForExternalTable, tblUpdateTime, rowCount, userInject, updateRows, - priority, partitionUpdateRows, enablePartition); + tableVersion, priority, partitionUpdateRows, enablePartition); } } 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 c2b20707f133e8..ece9daf25204c3 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 @@ -129,6 +129,7 @@ public class AnalysisManager implements Writable { public final Map>> highPriorityJobs = new LinkedHashMap<>(); public final Map>> midPriorityJobs = new LinkedHashMap<>(); public final Map>> lowPriorityJobs = new LinkedHashMap<>(); + public final Map>> veryLowPriorityJobs = new LinkedHashMap<>(); // Tracking running manually submitted async tasks, keep in mem only protected final ConcurrentMap> analysisJobIdToTaskMap = new ConcurrentHashMap<>(); @@ -381,7 +382,7 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) { } infoBuilder.setColName(stringJoiner.toString()); infoBuilder.setTaskIds(Lists.newArrayList()); - infoBuilder.setTblUpdateTime(System.currentTimeMillis()); + infoBuilder.setTblUpdateTime(table.getUpdateTime()); // Empty table row count is 0. Call fetchRowCount() when getRowCount() returns <= 0, // because getRowCount may return <= 0 if cached is not loaded. This is mainly for external table. long rowCount = StatisticsUtil.isEmptyTable(table, analysisMethod) ? 0 : @@ -389,6 +390,7 @@ public AnalysisInfo buildAnalysisJobInfo(AnalyzeTblStmt stmt) { infoBuilder.setRowCount(rowCount); TableStatsMeta tableStatsStatus = findTableStatsStatus(table.getId()); infoBuilder.setUpdateRows(tableStatsStatus == null ? 0 : tableStatsStatus.updatedRows.get()); + infoBuilder.setTableVersion(table instanceof OlapTable ? ((OlapTable) table).getVisibleVersion() : 0); infoBuilder.setPriority(JobPriority.MANUAL); infoBuilder.setPartitionUpdateRows(tableStatsStatus == null ? null : tableStatsStatus.partitionUpdateRows); infoBuilder.setEnablePartition(StatisticsUtil.enablePartitionAnalyze()); @@ -547,12 +549,15 @@ public List showAutoPendingJobs(ShowAutoAnalyzeJobsStmt result.addAll(getPendingJobs(highPriorityJobs, JobPriority.HIGH, tblName)); result.addAll(getPendingJobs(midPriorityJobs, JobPriority.MID, tblName)); result.addAll(getPendingJobs(lowPriorityJobs, JobPriority.LOW, tblName)); + result.addAll(getPendingJobs(veryLowPriorityJobs, JobPriority.VERY_LOW, tblName)); } else if (priority.equals(JobPriority.HIGH.name())) { result.addAll(getPendingJobs(highPriorityJobs, JobPriority.HIGH, tblName)); } else if (priority.equals(JobPriority.MID.name())) { result.addAll(getPendingJobs(midPriorityJobs, JobPriority.MID, tblName)); } else if (priority.equals(JobPriority.LOW.name())) { result.addAll(getPendingJobs(lowPriorityJobs, JobPriority.LOW, tblName)); + } else if (priority.equals(JobPriority.VERY_LOW.name())) { + result.addAll(getPendingJobs(veryLowPriorityJobs, JobPriority.VERY_LOW, tblName)); } return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java index 6cb2ced9286a65..78f51c2ac0c323 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColStatsMeta.java @@ -52,11 +52,15 @@ public class ColStatsMeta { @SerializedName("rowCount") public long rowCount; + @SerializedName("tv") + public long tableVersion; + @SerializedName("pur") public ConcurrentMap partitionUpdateRows = new ConcurrentHashMap<>(); - public ColStatsMeta(long updatedTime, AnalysisMethod analysisMethod, AnalysisType analysisType, JobType jobType, - long queriedTimes, long rowCount, long updatedRows, Map partitionUpdateRows) { + public ColStatsMeta(long updatedTime, AnalysisMethod analysisMethod, AnalysisType analysisType, + JobType jobType, long queriedTimes, long rowCount, long updatedRows, + long tableVersion, Map partitionUpdateRows) { this.updatedTime = updatedTime; this.analysisMethod = analysisMethod; this.analysisType = analysisType; @@ -64,6 +68,7 @@ public ColStatsMeta(long updatedTime, AnalysisMethod analysisMethod, AnalysisTyp this.queriedTimes.addAndGet(queriedTimes); this.updatedRows = updatedRows; this.rowCount = rowCount; + this.tableVersion = tableVersion; if (partitionUpdateRows != null) { this.partitionUpdateRows.putAll(partitionUpdateRows); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java index c3656b929279e6..df95b3cbede887 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/JobPriority.java @@ -21,5 +21,6 @@ public enum JobPriority { HIGH, MID, LOW, + VERY_LOW, MANUAL; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java index 38af43bef7316f..6c1fd1ba95951d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java @@ -26,7 +26,6 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; import org.apache.doris.common.util.MasterDaemon; -import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.statistics.AnalysisInfo.AnalysisMethod; import org.apache.doris.statistics.AnalysisInfo.JobType; @@ -37,7 +36,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.time.LocalTime; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -92,10 +90,11 @@ protected void runAfterCatalogReady() { } protected void collect() { - while (canCollect()) { + while (StatisticsUtil.canCollect()) { Pair>>, JobPriority> job = getJob(); if (job == null) { // No more job to process, break and sleep. + LOG.info("No auto analyze jobs to process."); break; } try { @@ -112,11 +111,6 @@ protected void collect() { } } - protected boolean canCollect() { - return StatisticsUtil.enableAutoAnalyze() - && StatisticsUtil.inAnalyzeTime(LocalTime.now(TimeUtils.getTimeZone().toZoneId())); - } - protected Pair>>, JobPriority> getJob() { AnalysisManager manager = Env.getServingEnv().getAnalysisManager(); Optional>>> job = fetchJobFromMap(manager.highPriorityJobs); @@ -128,7 +122,11 @@ protected Pair>>, JobPriority> getJob( return Pair.of(job.get(), JobPriority.MID); } job = fetchJobFromMap(manager.lowPriorityJobs); - return job.map(entry -> Pair.of(entry, JobPriority.LOW)).orElse(null); + if (job.isPresent()) { + return Pair.of(job.get(), JobPriority.LOW); + } + job = fetchJobFromMap(manager.veryLowPriorityJobs); + return job.map(tableNameSetEntry -> Pair.of(tableNameSetEntry, JobPriority.VERY_LOW)).orElse(null); } protected Optional>>> fetchJobFromMap( @@ -142,9 +140,13 @@ protected Optional>>> fetchJobFrom protected void processOneJob(TableIf table, Set> columns, JobPriority priority) throws DdlException { - // appendMvColumn(table, columns); appendAllColumns(table, columns); - columns = columns.stream().filter(c -> StatisticsUtil.needAnalyzeColumn(table, c)).collect(Collectors.toSet()); + columns = columns.stream().filter( + c -> StatisticsUtil.needAnalyzeColumn(table, c) || StatisticsUtil.isLongTimeColumn(table, c)) + .collect(Collectors.toSet()); + if (columns.isEmpty()) { + return; + } AnalysisInfo analyzeJob = createAnalyzeJobForTbl(table, columns, priority); if (analyzeJob == null) { return; @@ -178,15 +180,6 @@ protected void appendAllColumns(TableIf table, Set> columns } } - protected void appendMvColumn(TableIf table, Set columns) { - if (!(table instanceof OlapTable)) { - return; - } - OlapTable olapTable = (OlapTable) table; - Set mvColumns = olapTable.getMvColumns(false).stream().map(Column::getName).collect(Collectors.toSet()); - columns.addAll(mvColumns); - } - protected boolean supportAutoAnalyze(TableIf tableIf) { if (tableIf == null) { return false; @@ -248,9 +241,10 @@ protected AnalysisInfo createAnalyzeJobForTbl( .setTaskIds(new ArrayList<>()) .setLastExecTimeInMs(System.currentTimeMillis()) .setJobType(JobType.SYSTEM) - .setTblUpdateTime(System.currentTimeMillis()) + .setTblUpdateTime(table.getUpdateTime()) .setRowCount(rowCount) .setUpdateRows(tableStatsStatus == null ? 0 : tableStatsStatus.updatedRows.get()) + .setTableVersion(table instanceof OlapTable ? ((OlapTable) table).getVisibleVersion() : 0) .setPriority(priority) .setPartitionUpdateRows(tableStatsStatus == null ? null : tableStatsStatus.partitionUpdateRows) .setEnablePartition(StatisticsUtil.enablePartitionAnalyze()) @@ -275,4 +269,8 @@ protected void executeSystemAnalysisJob(AnalysisInfo jobInfo) future.get(); } } + + public boolean isReady() { + return waited; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java index b67d1cf947cb4e..4a3e93550fc9c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsJobAppender.java @@ -29,6 +29,7 @@ import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.statistics.util.StatisticsUtil; +import com.google.common.annotations.VisibleForTesting; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -46,7 +47,7 @@ public class StatisticsJobAppender extends MasterDaemon { private static final Logger LOG = LogManager.getLogger(StatisticsJobAppender.class); public static final long INTERVAL = 1000; - public static final int JOB_MAP_SIZE = 1000; + public static final int JOB_MAP_SIZE = 100; public static final int TABLE_BATCH_SIZE = 100; private long currentDbId = 0; @@ -70,6 +71,11 @@ protected void runAfterCatalogReady() { LOG.info("Stats table not available, skip"); return; } + if (Env.getCurrentEnv().getStatisticsAutoCollector() == null + || !Env.getCurrentEnv().getStatisticsAutoCollector().isReady()) { + LOG.info("Statistics auto collector not ready, skip"); + return; + } if (Env.isCheckpointThread()) { return; } @@ -81,7 +87,7 @@ protected void appendJobs() { appendColumnsToJobs(manager.highPriorityColumns, manager.highPriorityJobs); appendColumnsToJobs(manager.midPriorityColumns, manager.midPriorityJobs); if (StatisticsUtil.enableAutoAnalyzeInternalCatalog()) { - appendToLowJobs(manager.lowPriorityJobs); + appendToLowJobs(manager.lowPriorityJobs, manager.veryLowPriorityJobs); } } @@ -136,7 +142,8 @@ protected void appendColumnsToJobs(Queue columnQueue, Map>> jobs) { + protected void appendToLowJobs(Map>> lowPriorityJobs, + Map>> veryLowPriorityJobs) { if (System.currentTimeMillis() - lastRoundFinishTime < lowJobIntervalMs) { return; } @@ -162,27 +169,33 @@ protected void appendToLowJobs(Map>> jobs) { if (t.getBaseSchema().size() > StatisticsUtil.getAutoAnalyzeTableWidthThreshold()) { continue; } - Set> columnIndexPairs = t.getColumnIndexPairs( - t.getSchemaAllIndexes(false).stream() - .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) - .map(Column::getName).collect(Collectors.toSet())) + Set columns = t.getSchemaAllIndexes(false).stream() + .filter(c -> !StatisticsUtil.isUnsupportedType(c.getType())) + .map(Column::getName).collect(Collectors.toSet()); + Set> columnIndexPairs = t.getColumnIndexPairs(columns) .stream().filter(p -> StatisticsUtil.needAnalyzeColumn(t, p)) .collect(Collectors.toSet()); - if (columnIndexPairs.isEmpty()) { - continue; - } TableName tableName = new TableName(t.getDatabase().getCatalog().getName(), t.getDatabase().getFullName(), t.getName()); - synchronized (jobs) { - // If job map reach the upper limit, stop adding new jobs. - if (!jobs.containsKey(tableName) && jobs.size() >= JOB_MAP_SIZE) { - LOG.info("Low job map full."); + // Append to low job map first. + if (!columnIndexPairs.isEmpty()) { + boolean appended = doAppend(lowPriorityJobs, columnIndexPairs, tableName); + // If low job map is full, stop this iteration. + if (!appended) { + LOG.debug("Low Priority job map is full."); return; } - if (jobs.containsKey(tableName)) { - jobs.get(tableName).addAll(columnIndexPairs); - } else { - jobs.put(tableName, columnIndexPairs); + } else { + // Append to very low job map. + columnIndexPairs = t.getColumnIndexPairs(columns) + .stream().filter(p -> StatisticsUtil.isLongTimeColumn(t, p)) + .collect(Collectors.toSet()); + if (!columnIndexPairs.isEmpty()) { + boolean appended = doAppend(veryLowPriorityJobs, columnIndexPairs, tableName); + // If very low job map is full, simply ignore it and go to the next table. + if (!appended) { + LOG.debug("Very low Priority job map is full."); + } } } currentTableId = t.getId(); @@ -200,6 +213,23 @@ protected void appendToLowJobs(Map>> jobs) { lastRoundFinishTime = System.currentTimeMillis(); } + @VisibleForTesting + public boolean doAppend(Map>> jobMap, + Set> columnIndexPairs, + TableName tableName) { + synchronized (jobMap) { + if (!jobMap.containsKey(tableName) && jobMap.size() >= JOB_MAP_SIZE) { + return false; + } + if (jobMap.containsKey(tableName)) { + jobMap.get(tableName).addAll(columnIndexPairs); + } else { + jobMap.put(tableName, columnIndexPairs); + } + } + return true; + } + // For unit test only. public void setLastRoundFinishTime(long value) { lastRoundFinishTime = value; diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java index ba23ab84dc7a32..ac4704b54c6aae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsRepository.java @@ -379,9 +379,8 @@ public static void alterColumnStatistics(AlterColumnStatsStmt alterColumnStatsSt objects.catalog.getId(), objects.db.getId(), objects.table.getId(), indexId, colName, null, columnStatistic); Env.getCurrentEnv().getStatisticsCache().syncColStats(data); - long timestamp = System.currentTimeMillis(); AnalysisInfo mockedJobInfo = new AnalysisInfoBuilder() - .setTblUpdateTime(timestamp) + .setTblUpdateTime(objects.table.getUpdateTime()) .setColName("") .setRowCount((long) Double.parseDouble(rowCount)) .setJobColumns(Sets.newHashSet()) diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java index 6a7f2933996d88..4ebdb019f27e36 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/TableStatsMeta.java @@ -80,6 +80,9 @@ public class TableStatsMeta implements Writable, GsonPostProcessable { @SerializedName("updateTime") public long updatedTime; + @SerializedName("lat") + public long lastAnalyzeTime; + @SerializedName("colNameToColStatsMeta") private ConcurrentMap deprecatedColNameToColStatsMeta = new ConcurrentHashMap<>(); @@ -160,6 +163,7 @@ public Set> analyzeColumns() { public void update(AnalysisInfo analyzedJob, TableIf tableIf) { updatedTime = analyzedJob.tblUpdateTime; + lastAnalyzeTime = analyzedJob.createTime; if (analyzedJob.userInject) { userInjected = true; } @@ -168,14 +172,16 @@ public void update(AnalysisInfo analyzedJob, TableIf tableIf) { if (colStatsMeta == null) { colToColStatsMeta.put(colPair, new ColStatsMeta(analyzedJob.createTime, analyzedJob.analysisMethod, analyzedJob.analysisType, analyzedJob.jobType, 0, analyzedJob.rowCount, - analyzedJob.updateRows, analyzedJob.enablePartition ? analyzedJob.partitionUpdateRows : null)); + analyzedJob.updateRows, analyzedJob.tableVersion, + analyzedJob.enablePartition ? analyzedJob.partitionUpdateRows : null)); } else { - colStatsMeta.updatedTime = analyzedJob.tblUpdateTime; + colStatsMeta.updatedTime = analyzedJob.createTime; colStatsMeta.analysisType = analyzedJob.analysisType; colStatsMeta.analysisMethod = analyzedJob.analysisMethod; colStatsMeta.jobType = analyzedJob.jobType; colStatsMeta.updatedRows = analyzedJob.updateRows; colStatsMeta.rowCount = analyzedJob.rowCount; + colStatsMeta.tableVersion = analyzedJob.tableVersion; if (analyzedJob.enablePartition) { if (colStatsMeta.partitionUpdateRows == null) { colStatsMeta.partitionUpdateRows = new ConcurrentHashMap<>(); 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 a9c1612eb48b9c..dd037617ba6e30 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 @@ -53,6 +53,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.InternalCatalog; @@ -1012,12 +1013,6 @@ public static boolean needAnalyzeColumn(TableIf table, Pair colu if (columnStatsMeta == null) { return true; } - // Column hasn't been analyzed for longer than config interval. - if (Config.auto_analyze_interval_seconds > 0 - && System.currentTimeMillis() - columnStatsMeta.updatedTime - > Config.auto_analyze_interval_seconds * 1000) { - return true; - } // Partition table partition stats never been collected. if (StatisticsUtil.enablePartitionAnalyze() && table.isPartitionedTable() && columnStatsMeta.partitionUpdateRows == null) { @@ -1072,7 +1067,7 @@ public static boolean needAnalyzeColumn(TableIf table, Pair colu } // External is hard to calculate change rate, use time interval to control analyze frequency. return System.currentTimeMillis() - - tableStatsStatus.updatedTime > StatisticsUtil.getExternalTableAutoAnalyzeIntervalInMillis(); + - tableStatsStatus.lastAnalyzeTime > StatisticsUtil.getExternalTableAutoAnalyzeIntervalInMillis(); } } @@ -1127,4 +1122,47 @@ public static boolean needAnalyzePartition(OlapTable table, TableStatsMeta table } return false; } + + // This function return true means the column hasn't been analyzed for longer than the configured time. + public static boolean isLongTimeColumn(TableIf table, Pair column) { + if (column == null) { + return false; + } + if (!table.autoAnalyzeEnabled()) { + return false; + } + if (!(table instanceof OlapTable)) { + return false; + } + AnalysisManager manager = Env.getServingEnv().getAnalysisManager(); + TableStatsMeta tblStats = manager.findTableStatsStatus(table.getId()); + // Table never been analyzed, skip it for higher priority jobs. + if (tblStats == null) { + LOG.warn("Table stats is null."); + return false; + } + ColStatsMeta columnStats = tblStats.findColumnStatsMeta(column.first, column.second); + if (columnStats == null) { + // Column never been analyzed, skip it for higher priority jobs. + return false; + } + // User injected column stats, don't do auto analyze, avoid overwrite user injected stats. + if (tblStats.userInjected) { + return false; + } + boolean isLongTime = Config.auto_analyze_interval_seconds > 0 + && System.currentTimeMillis() - columnStats.updatedTime > Config.auto_analyze_interval_seconds * 1000; + if (!isLongTime) { + return false; + } + // For olap table, if the table visible version and row count doesn't change since last analyze, + // we don't need to analyze it because its data is not changed. + OlapTable olapTable = (OlapTable) table; + return olapTable.getVisibleVersion() != columnStats.tableVersion + || olapTable.getRowCount() != columnStats.rowCount; + } + + public static boolean canCollect() { + return enableAutoAnalyze() && inAnalyzeTime(LocalTime.now(TimeUtils.getTimeZone().toZoneId())); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java index 67b44adc565534..f14f5113d8feea 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java @@ -45,7 +45,7 @@ public void setUp() { @Test public void testNormal() throws UserException, AnalysisException { - DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true); + DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), false); stmt.analyze(analyzer); Assert.assertEquals("test", stmt.getCtlName()); @@ -53,6 +53,16 @@ public void testNormal() throws UserException, AnalysisException { Assert.assertEquals("DROP DATABASE `test`", stmt.toString()); } + @Test + public void testForce() throws UserException, AnalysisException { + DropDbStmt stmt = new DropDbStmt(false, new DbName("test", "test"), true); + + stmt.analyze(analyzer); + Assert.assertEquals("test", stmt.getCtlName()); + Assert.assertEquals("test", stmt.getDbName()); + Assert.assertEquals("DROP DATABASE `test` FORCE", stmt.toString()); + } + @Test(expected = AnalysisException.class) public void testFailed() throws UserException, AnalysisException { DropDbStmt stmt = new DropDbStmt(false, new DbName("", ""), true); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java index da6d5b8d4c44d0..437e54f58f20e6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java @@ -72,12 +72,13 @@ public void testNormal() throws UserException, AnalysisException { stmt.analyze(analyzer); Assert.assertEquals("db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); - Assert.assertEquals("DROP TABLE `db1`.`table1`", stmt.toString()); + // one with force. + Assert.assertEquals("DROP TABLE `db1`.`table1` FORCE", stmt.toString()); } @Test public void testDefaultNormal() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, noDbTbl, true); + DropTableStmt stmt = new DropTableStmt(false, noDbTbl, false); stmt.analyze(analyzer); Assert.assertEquals("testDb", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java index 353fbad9fabfc8..0faf4a8f34e8a8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java @@ -32,7 +32,7 @@ protected void runBeforeAll() throws Exception { createDatabase("test"); useDatabase("test"); createTable("create table table1\n" - + "(k1 int comment 'test column k1', k2 int comment 'test column k2') comment 'test table1' " + + "(k1 int comment 'test column k1', k2 int comment 'test column k2', `timestamp` DATE NOT NULL COMMENT '[''0000-01-01'', ''9999-12-31'']') comment 'test table1' " + "PARTITION BY RANGE(`k1`)\n" + "(\n" + " PARTITION `p01` VALUES LESS THAN (\"10\"),\n" @@ -48,10 +48,20 @@ public void testNormal() throws Exception { String sql = "show create table table1"; ShowResultSet showResultSet = showCreateTable(sql); String showSql = showResultSet.getResultRows().get(0).get(1); - Assertions.assertTrue(showSql.contains("`k1` int NULL COMMENT 'test column k1'")); + Assertions.assertTrue(showSql.contains("`k1` int NULL COMMENT \"test column k1\"")); Assertions.assertTrue(showSql.contains("COMMENT 'test table1'")); } + @Test + public void testColumnComment() throws Exception { + String sql = "show create table table1"; + ShowResultSet showResultSet = showCreateTable(sql); + String showSql = showResultSet.getResultRows().get(0).get(1); + Assertions.assertTrue(showSql.contains("`k1` int NULL COMMENT \"test column k1\"")); + Assertions.assertTrue(showSql.contains("`k2` int NULL COMMENT \"test column k2\"")); + Assertions.assertTrue(showSql.contains("`timestamp` date NOT NULL COMMENT \"['0000-01-01', '9999-12-31']\"")); + } + @Test public void testBrief() throws Exception { String sql = "show brief create table table1"; diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableWithBloomFilterIndexTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableWithBloomFilterIndexTest.java new file mode 100644 index 00000000000000..54787ee8eefe92 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableWithBloomFilterIndexTest.java @@ -0,0 +1,905 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.catalog; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ExceptionChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +import java.util.UUID; + +public class CreateTableWithBloomFilterIndexTest extends TestWithFeService { + private static String runningDir = "fe/mocked/CreateTableWithBloomFilterIndexTest/" + + UUID.randomUUID().toString() + "/"; + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + } + + @Test + public void testCreateTableWithTinyIntBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "TINYINT is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_tinyint_bf (\n" + + "k1 TINYINT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithSupportedIntBloomFilterIndex() throws Exception { + // smallint + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_smallint_bf (\n" + + "k1 SMALLINT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + + // int + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_int_bf (\n" + + "k1 INT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + + // bigint + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_bigint_bf (\n" + + "k1 BIGINT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + + // largeint + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_largeint_bf (\n" + + "k1 LARGEINT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithFloatBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "FLOAT is not supported in bloom filter index. invalid column: k2", + () -> createTable("CREATE TABLE test.tbl_float_bf (\n" + + "k1 INT, \n" + + "k2 FLOAT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k2\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithDoubleBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "DOUBLE is not supported in bloom filter index. invalid column: k2", + () -> createTable("CREATE TABLE test.tbl_double_bf (\n" + + "k1 INT, \n" + + "k2 DOUBLE, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k2\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithDecimalBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_decimal_bf (\n" + + "k1 DECIMAL(10,2), \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithCharBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_char_bf (\n" + + "k1 CHAR(20), \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithVarcharBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_varchar_bf (\n" + + "k1 VARCHAR(20), \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithTextBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_text_bf (\n" + + "k1 INT, \n" + + "k2 TEXT, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k2\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithDecimalV3BloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_decimalv3_bf (\n" + + "k1 DECIMALV3(10,2), \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithIPv4BloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_ipv4_bf (\n" + + "k1 IPV4, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithIPv6BloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_ipv6_bf (\n" + + "k1 IPV6, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithDateBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_date_bf (\n" + + "k1 DATE, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithDateTimeBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_datetime_bf (\n" + + "k1 DATETIME, \n" + + "v1 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithCharNgramBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_char_ngram_bf (\n" + + "k1 CHAR(20), \n" + + "v1 INT,\n" + + "INDEX idx_k1_ngram (k1) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"3\", \"bf_size\"=\"1024\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithVarcharNgramBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_varchar_ngram_bf (\n" + + "k1 VARCHAR(50), \n" + + "v1 INT,\n" + + "INDEX idx_k1_ngram (k1) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"3\", \"bf_size\"=\"1024\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithStringNgramBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_string_ngram_bf (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"3\", \"bf_size\"=\"1024\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithArrayNumericBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "ARRAY is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_array_numeric_bf (\n" + + "v1 INT,\n" + + "k1 ARRAY\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithArrayDateBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "ARRAY is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_array_date_bf (\n" + + "v1 INT,\n" + + "k1 ARRAY\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithArrayStringNgramBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + " ARRAY is not supported in ngram_bf index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_array_string_ngram_bf (\n" + + "v1 INT,\n" + + "k1 ARRAY,\n" + + "INDEX idx_k1_ngram (k1) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"3\", \"bf_size\"=\"1024\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithMapBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "MAP is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_map_bf (\n" + + "v1 INT,\n" + + "k1 MAP\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithStructBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "STRUCT is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_struct_bf (\n" + + "v1 INT,\n" + + "k1 STRUCT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithJsonBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + " JSON is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_json_bf (\n" + + "v1 INT,\n" + + "k1 JSON\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateTableWithHllBloomFilterIndex() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + " HLL is not supported in bloom filter index. invalid column: k1", + () -> createTable("CREATE TABLE test.tbl_hll_bf (\n" + + "v1 INT,\n" + + "k1 HLL\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(v1)\n" + + "DISTRIBUTED BY HASH(v1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateMowTableWithBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_mow_bf (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "UNIQUE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"enable_unique_key_merge_on_write\" = \"true\"\n" + + ");")); + } + + @Test + public void testCreateDuplicateTableWithBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_duplicate_bf (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testCreateMorTableWithBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_mor_bf (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "UNIQUE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"enable_unique_key_merge_on_write\" = \"false\"\n" + + ");")); + } + + @Test + public void testCreateAggTableWithBloomFilterIndex() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_agg_bf (\n" + + "k1 INT, \n" + + "v1 INT SUM\n" + + ") ENGINE=OLAP\n" + + "AGGREGATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsValidCharacters() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_bf_valid_chars (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1,v1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsInvalidCharacters() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: k1;v1", + () -> createTable("CREATE TABLE test.tbl_bf_invalid_chars (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1;v1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterFppValidInput() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_bf_fpp_valid (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"bloom_filter_fpp\" = \"0.05\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterFppInvalidInput() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter fpp should in [1.0E-4, 0.05]", + () -> createTable("CREATE TABLE test.tbl_bf_fpp_invalid (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"bloom_filter_fpp\" = \"-0.05\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testNgramBloomFilterGramSizeValidInput() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_ngram_gramsize_valid (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"4\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testNgramBloomFilterGramSizeInvalidInput() { + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "'gram_size' should be an integer between 1 and 255", + () -> createTable("CREATE TABLE test.tbl_ngram_gramsize_invalid (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"-1\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testNgramBloomFilterGramSizeInvalidInput256() { + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "'gram_size' should be an integer between 1 and 255", + () -> createTable("CREATE TABLE test.tbl_ngram_gram_size_invalid (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"gram_size\"=\"256\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testNgramBloomFilterBfSizeValidInput() throws Exception { + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "'bf_size' should be an integer between 64 and 65535", + () -> createTable("CREATE TABLE test.tbl_ngram_bfsize_valid (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"bf_size\"=\"256000000\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testNgramBloomFilterBfSizeInvalidInput() { + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "'bf_size' should be an integer between 64 and 65535", + () -> createTable("CREATE TABLE test.tbl_ngram_bfsize_invalid (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"bf_size\"=\"-256000000\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testNgramBloomFilterBfSizeInvalidInput65536() { + ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, + "'bf_size' should be an integer between 64 and 65535", + () -> createTable("CREATE TABLE test.tbl_ngram_bf_size_invalid (\n" + + "k1 INT, \n" + + "k2 STRING, \n" + + "v1 INT,\n" + + "INDEX idx_k2_ngram (k2) USING NGRAM_BF PROPERTIES(\"bf_size\"=\"65536\")\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsDuplicated() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Reduplicated bloom filter column: k1", + () -> createTable("CREATE TABLE test.tbl_bf_duplicated_columns (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1,k1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnDoesNotExist() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: k3", + () -> createTable("CREATE TABLE test.tbl_bf_column_not_exist (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k3\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnInvalidType() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "BOOLEAN is not supported in bloom filter index. invalid column: k2", + () -> createTable("CREATE TABLE test.tbl_bf_invalid_type (\n" + + "k1 INT, \n" + + "k2 BOOLEAN,\n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k2\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnNonKeyInAggKeys() throws Exception { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter index should only be used in columns of UNIQUE_KEYS/DUP_KEYS table or key columns of AGG_KEYS table. invalid column: v1", + () -> createTable("CREATE TABLE test.tbl_bf_nonkey_in_agg (\n" + + "k1 INT, \n" + + "v1 INT SUM\n" + + ") ENGINE=OLAP\n" + + "AGGREGATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterFppNotDouble() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter fpp is not Double", + () -> createTable("CREATE TABLE test.tbl_bf_fpp_not_double (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"bloom_filter_fpp\" = \"abc\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterFppOutOfRange() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter fpp should in [1.0E-4, 0.05]", + () -> createTable("CREATE TABLE test.tbl_bf_fpp_out_of_range (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"bloom_filter_fpp\" = \"0.1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterFppBelowMin() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter fpp should in [1.0E-4, 0.05]", + () -> createTable("CREATE TABLE test.tbl_bf_fpp_below_min (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"v1\",\n" + + "\"bloom_filter_fpp\" = \"1e-5\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsEmptyString() throws Exception { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Unknown properties: {bloom_filter_columns=}", + () -> createTable("CREATE TABLE test.tbl_bf_empty_columns (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsOnlyCommas() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Unknown properties: {bloom_filter_columns=,,,}", + () -> createTable("CREATE TABLE test.tbl_bf_only_commas (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \",,,\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsNonExistingColumns() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: k3", + () -> createTable("CREATE TABLE test.tbl_bf_non_existing_columns (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20),\n" + + "k2 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k2,k3\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsWithSpecialCharacters() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: k1@", + () -> createTable("CREATE TABLE test.tbl_bf_special_chars (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20),\n" + + "k2 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1@,v1#\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsWithDifferentCase() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_bf_different_case (\n" + + "k1 INT, \n" + + "V1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(K1)\n" + + "DISTRIBUTED BY HASH(K1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"k1,v1\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsWithSpaces() throws Exception { + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.tbl_bf_columns_with_spaces (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \" k1 , v1 \",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsWithLongColumnName() throws Exception { + StringBuilder sb = new StringBuilder("k"); + for (int i = 0; i < 1000; i++) { + sb.append('1'); + } + String longColumnName = sb.toString(); + + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: " + longColumnName, + () -> createTable("CREATE TABLE test.tbl_bf_long_column_name (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20),\n" + + "k2 INT\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"" + longColumnName + "\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsWithUnicodeCharacters() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: 名字", + () -> createTable("CREATE TABLE test.tbl_bf_unicode_columns (\n" + + "k1 INT, \n" + + "name VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \"名字\",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } + + @Test + public void testBloomFilterColumnsWithNullOrWhitespace() { + ExceptionChecker.expectThrowsWithMsg(DdlException.class, + "Bloom filter column does not exist in table. invalid column: ", + () -> createTable("CREATE TABLE test.tbl_bf_null_or_whitespace (\n" + + "k1 INT, \n" + + "v1 VARCHAR(20)\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"bloom_filter_columns\" = \" , \",\n" + + "\"replication_num\" = \"1\"\n" + + ");")); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java index affd5a4cc2c397..c6b45c0b136e61 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java @@ -268,4 +268,10 @@ public void testSerialization() throws Exception { dis.close(); Files.deleteIfExists(path); } + + @Test + public void testMaxValueToSql() throws Exception { + PartitionKey key = PartitionKey.createInfinityPartitionKey(allColumns, true); + Assert.assertEquals("(MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE, MAXVALUE)", key.toSql()); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java b/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java new file mode 100644 index 00000000000000..8d4cac9b6ad9f4 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/httpv2/GetLogFileActionTest.java @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.httpv2; + +import org.apache.doris.common.Config; +import org.apache.doris.httpv2.rest.GetLogFileAction; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +public class GetLogFileActionTest { + + @TempDir + public File tempDir; + + @BeforeAll + public static void before() { + File tempDir = new File("test/audit.log"); + tempDir.mkdir(); + Config.audit_log_dir = tempDir.getAbsolutePath(); + } + + @Test + public void testCheckAuditLogFileName() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + //private method checkAuditLogFileName + GetLogFileAction action = new GetLogFileAction(); + Method method = GetLogFileAction.class.getDeclaredMethod("checkAuditLogFileName", String.class); + method.setAccessible(true); + method.invoke(action, "audit.log"); + method.invoke(action, "fe.audit.log.20241104-1"); + Assertions.assertThrows(InvocationTargetException.class, () -> method.invoke(action, "../etc/passwd")); + Assertions.assertThrows(InvocationTargetException.class, () -> method.invoke(action, + "fe.audit.log.20241104-1/../../etc/passwd")); + Assertions.assertThrows(InvocationTargetException.class, + () -> method.invoke(action, "fe.audit.log.20241104-1; rm -rf /")); + + + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/BetweenTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/BetweenTest.java new file mode 100644 index 00000000000000..c9ef4fbee27cbe --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/BetweenTest.java @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.parser; + +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class BetweenTest { + private static final NereidsParser PARSER = new NereidsParser(); + + @Test + public void testBetween() { + String expression = "A between 1 and 1"; // + Expression result = PARSER.parseExpression(expression); + Assertions.assertInstanceOf(EqualTo.class, result); + + expression = "A between 1 and 2"; + result = PARSER.parseExpression(expression); + Assertions.assertInstanceOf(And.class, result); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index f824a40eda6474..45e1190412d0a4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -248,7 +248,7 @@ protected void runBeforeAll() throws Exception { + "\"replication_allocation\" = \"tag.location.default: 1\"\n" + ");\n"); // Should not make scan to empty relation when the table used by materialized view has no data - connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION,ELIMINATE_GROUP_BY_KEY_BY_UNIFORM"); } // Test when join both side are all partition table and partition column name is same diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java new file mode 100644 index 00000000000000..78d8034e3fdfed --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyByUniformTest.java @@ -0,0 +1,250 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import org.junit.jupiter.api.Test; + +public class EliminateGroupByKeyByUniformTest extends TestWithFeService implements MemoPatternMatchSupported { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + createTable("create table test.eli_gbk_by_uniform_t(a int null, b int not null," + + "c varchar(10) null, d date, dt datetime)\n" + + "distributed by hash(a) properties('replication_num' = '1');"); + connectContext.setDatabase("test"); + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + } + + @Test + void testEliminateByFilter() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a),b from eli_gbk_by_uniform_t where a = 1 group by a,b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("b"))); + + } + + @Test + void testNotEliminateWhenOnlyOneGbyKey() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a) from eli_gbk_by_uniform_t where a = 1 group by a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + + } + + @Test + void testEliminateByProjectConst() { + PlanChecker.from(connectContext) + .analyze("select sum(c1), c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 ") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c3"))); + } + + @Test + void testEliminateByProjectUniformSlot() { + PlanChecker.from(connectContext) + .analyze("select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t group by c1,c2,c3") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c3"))); + } + + @Test + void testEliminateDate() { + PlanChecker.from(connectContext) + .analyze("select d, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where d = '2023-01-06' group by d,a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + } + + @Test + void testSaveOneExpr() { + PlanChecker.from(connectContext) + .analyze("select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=100 group by a, b,'abc'") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("a"))); + } + + @Test + void testSaveOneExprProjectConst() { + PlanChecker.from(connectContext) + .analyze("select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> + agg.getGroupByExpressions().size() == 1 + && agg.getGroupByExpressions().get(0).toSql().equals("c2"))); + } + + @Test + void testNotRewriteWhenHasRepeat() { + PlanChecker.from(connectContext) + .analyze("select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by grouping sets((c2),(c3)) order by 1;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testInnerJoin() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testLeftJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testRightJoinOnConditionNullableSideFilterNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testRightJoinOnConditionNonNullableSideFilterNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 3)); + } + + @Test + void testRightJoinWhereConditionToInnerRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t2.b,t2.c;") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftSemiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftSemiJoinRetainOneSlotInGroupBy() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightSemiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightSemiJoinRetainOneSlotInGroupBy() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testLeftAntiJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testLeftAntiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t1.c") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } + + @Test + void testRightAntiJoinOnConditionNotRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.a") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 2)); + } + + @Test + void testRightAntiJoinWhereConditionRewrite() { + PlanChecker.from(connectContext) + .analyze("select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t2.b=100 group by t2.b,t2.c") + .rewrite() + .printlnTree() + .matches(logicalAggregate().when(agg -> agg.getGroupByExpressions().size() == 1)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java index 5a9e15cf4774d1..103e074c73bfd5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByKeyTest.java @@ -111,7 +111,7 @@ void testProjectAlias() { .rewrite() .printlnTree() .matches(logicalAggregate().when(agg -> - agg.getGroupByExpressions().size() == 2)); + agg.getGroupByExpressions().size() == 1)); PlanChecker.from(connectContext) .analyze("select id as c, name as n from t1 group by name, id") .rewrite() @@ -123,7 +123,7 @@ void testProjectAlias() { .rewrite() .printlnTree() .matches(logicalAggregate().when(agg -> - agg.getGroupByExpressions().size() == 2)); + agg.getGroupByExpressions().size() == 1)); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java index e7940cdfb9b519..385657cd3f229e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java @@ -28,6 +28,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.nereids.sqltest.SqlTestBase; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; import org.apache.doris.nereids.trees.expressions.InPredicate; @@ -51,8 +52,9 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Objects; -class PruneOlapScanTabletTest implements MemoPatternMatchSupported { +class PruneOlapScanTabletTest extends SqlTestBase implements MemoPatternMatchSupported { @Test void testPruneOlapScanTablet(@Mocked OlapTable olapTable, @@ -154,4 +156,21 @@ void testPruneOlapScanTablet(@Mocked OlapTable olapTable, ) ); } + + @Test + void testPruneOlapScanTabletWithManually() { + String sql = "select * from T4 TABLET(110) where id > 8"; + PlanChecker.from(connectContext) + .analyze(sql) + .applyTopDown(new PruneOlapScanTablet()) + .matches( + logicalFilter( + logicalOlapScan().when(s -> + Objects.equals(s.getSelectedTabletIds(), Lists.newArrayList(110L)) + && Objects.equals(s.getManuallySpecifiedTabletIds(), + Lists.newArrayList(110L)) + ) + ) + ); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java index 58ab7fbe9e925f..cffe91045d0ab2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownMinMaxSumThroughJoinTest.java @@ -323,11 +323,11 @@ void testSingleCountStar() { .applyTopDown(new PushDownAggThroughJoinOneSide()) .printlnTree() .matches( - logicalAggregate( - logicalJoin( - logicalOlapScan(), + logicalJoin( + logicalAggregate( logicalOlapScan() - ) + ), + logicalOlapScan() ) ); } @@ -346,11 +346,9 @@ void testBothSideCountAndCountStar() { PlanChecker.from(MemoTestUtils.createConnectContext(), plan) .applyTopDown(new PushDownAggThroughJoinOneSide()) .matches( - logicalAggregate( - logicalJoin( - logicalOlapScan(), - logicalOlapScan() - ) + logicalJoin( + logicalAggregate(logicalOlapScan()), + logicalAggregate(logicalOlapScan()) ) ); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java index f5b301dcc49b25..9ebbc22a85c826 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/SqlTestBase.java @@ -87,6 +87,9 @@ protected void runBeforeAll() throws Exception { + " score bigint\n" + ")\n" + "DUPLICATE KEY(id)\n" + + "AUTO PARTITION BY LIST(`id`)\n" + + "(\n" + + ")\n" + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + "PROPERTIES (\n" + " \"replication_num\" = \"1\"\n" diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java index 60f6e19faab635..0c54f8fad5a107 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java @@ -130,7 +130,7 @@ public void test1() { Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Random); // Check get tables TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true); + Sets.newHashSet(TableType.OLAP), true, connectContext); physicalPlan.accept(TableCollector.INSTANCE, collectorContext); Set expectedTables = new HashSet<>(); expectedTables.add("table1"); @@ -159,7 +159,7 @@ public void test2() { Assertions.assertTrue(nondeterministicFunctionSet.get(1) instanceof Random); // Check get tables TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true); + Sets.newHashSet(TableType.OLAP), true, connectContext); physicalPlan.accept(TableCollector.INSTANCE, collectorContext); Set expectedTables = new HashSet<>(); expectedTables.add("table1"); @@ -196,7 +196,7 @@ public BitSet getDisableNereidsRules() { Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Uuid); // Check get tables TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true); + Sets.newHashSet(TableType.OLAP), true, connectContext); physicalPlan.accept(TableCollector.INSTANCE, collectorContext); Set expectedTables = new HashSet<>(); expectedTables.add("table1"); @@ -210,7 +210,7 @@ public BitSet getDisableNereidsRules() { TableCollectorContext collectorContextWithNoExpand = new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.OLAP), - false); + false, connectContext); physicalPlan.accept(TableCollector.INSTANCE, collectorContextWithNoExpand); Set expectedTablesWithNoExpand = new HashSet<>(); expectedTablesWithNoExpand.add("table1"); @@ -222,7 +222,7 @@ public BitSet getDisableNereidsRules() { expectedTablesWithNoExpand); TableCollectorContext mvCollectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.MATERIALIZED_VIEW), true); + Sets.newHashSet(TableType.MATERIALIZED_VIEW), true, connectContext); physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContext); Set expectedMvs = new HashSet<>(); expectedMvs.add("mv1"); @@ -234,7 +234,7 @@ public BitSet getDisableNereidsRules() { TableCollectorContext mvCollectorContextWithNoExpand = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.MATERIALIZED_VIEW), false); + Sets.newHashSet(TableType.MATERIALIZED_VIEW), false, connectContext); physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContextWithNoExpand); Set expectedMvsWithNoExpand = new HashSet<>(); expectedMvsWithNoExpand.add("mv1"); @@ -246,7 +246,7 @@ public BitSet getDisableNereidsRules() { TableCollectorContext allTableTypeWithExpand = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.values()), true); + Sets.newHashSet(TableType.values()), true, connectContext); physicalPlan.accept(TableCollector.INSTANCE, allTableTypeWithExpand); // when collect in plan with expand, should collect table which is expended Set expectedTablesWithExpand = new HashSet<>(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/LimitUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/LimitUtilsTest.java new file mode 100644 index 00000000000000..012fbad18a5ddb --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/LimitUtilsTest.java @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.qe; + + +import org.apache.doris.common.Status; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.function.Consumer; + +public class LimitUtilsTest { + + private static int res = 0; + + @Test + public void testUpperBound() { + Consumer cancelFunc = batch -> res = 666; + RowBatch rowBatch = new RowBatch(); + rowBatch.setEos(false); + // - no limit + Assert.assertFalse(LimitUtils.cancelIfReachLimit(rowBatch, 0, 10, cancelFunc)); + Assert.assertFalse(rowBatch.isEos()); + Assert.assertEquals(0, res); + + // - not reach limit + Assert.assertFalse(LimitUtils.cancelIfReachLimit(rowBatch, 10, 1, cancelFunc)); + Assert.assertFalse(rowBatch.isEos()); + Assert.assertEquals(0, res); + + // - reach limit + Assert.assertTrue(LimitUtils.cancelIfReachLimit(rowBatch, 10, 10, cancelFunc)); + Assert.assertTrue(rowBatch.isEos()); + Assert.assertEquals(666, res); + + // - reach limit + res = 0; + rowBatch.setEos(false); + Assert.assertTrue(LimitUtils.cancelIfReachLimit(rowBatch, 10, 100, cancelFunc)); + Assert.assertTrue(rowBatch.isEos()); + Assert.assertEquals(666, res); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java index e3255ab23a0381..5b890795f01076 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsJobAppenderTest.java @@ -33,6 +33,8 @@ import org.apache.doris.statistics.util.StatisticsUtil; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import mockit.Mock; import mockit.MockUp; import org.junit.jupiter.api.Assertions; @@ -178,13 +180,21 @@ public Set> getColumnIndexPairs(Set columns) { } }; - Map>> testMap = new HashMap<>(); + new MockUp() { + @Mock + public boolean needAnalyzeColumn(TableIf table, Pair column) { + return true; + } + }; + + Map>> testLowMap = new HashMap<>(); + Map>> testVeryLowMap = new HashMap<>(); StatisticsJobAppender appender = new StatisticsJobAppender(); - appender.appendToLowJobs(testMap); - Assertions.assertEquals(100, testMap.size()); - testMap.clear(); - appender.appendToLowJobs(testMap); - Assertions.assertEquals(40, testMap.size()); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(100, testLowMap.size()); + testLowMap.clear(); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(40, testLowMap.size()); for (int i = 0; i < StatisticsJobAppender.JOB_MAP_SIZE; i++) { Database db = new Database(id++, "testDb" + i); @@ -198,38 +208,93 @@ public Set> getColumnIndexPairs(Set columns) { db.createTableWithLock(table2, true, false); } - testMap.clear(); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); + testLowMap.clear(); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); + appender.appendToLowJobs(testLowMap, testVeryLowMap); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); + appender.appendToLowJobs(testLowMap, testVeryLowMap); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(StatisticsJobAppender.JOB_MAP_SIZE, testLowMap.size()); + } + + @Test + public void testAppendQueryColumnToVeryLowJobMap() throws DdlException { + InternalCatalog testCatalog = new InternalCatalog(); + int id = 10; + for (int i = 0; i < 70; i++) { + Database db = new Database(id++, "testDb" + i); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + OlapTable table2 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + db.createTableWithLock(table1, true, false); + db.createTableWithLock(table2, true, false); + } + + new MockUp() { + @Mock + public InternalCatalog getCurrentInternalCatalog() { + return testCatalog; + } + }; + + new MockUp() { + @Mock + public List getBaseSchema() { + return Lists.newArrayList(); + } + + @Mock + public Set> getColumnIndexPairs(Set columns) { + return Collections.singleton(Pair.of("mockIndex", "mockColumn")); + } + }; + + new MockUp() { + @Mock + public boolean needAnalyzeColumn(TableIf table, Pair column) { + return false; + } + + @Mock + public boolean isLongTimeColumn(TableIf table, Pair column) { + return true; + } + }; + + Map>> testLowMap = new HashMap<>(); + Map>> testVeryLowMap = new HashMap<>(); + StatisticsJobAppender appender = new StatisticsJobAppender(); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(100, testVeryLowMap.size()); + testVeryLowMap.clear(); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(40, testVeryLowMap.size()); + + for (int i = 0; i < StatisticsJobAppender.JOB_MAP_SIZE; i++) { + Database db = new Database(id++, "testDb" + i); + testCatalog.unprotectCreateDb(db); + Column column1 = new Column("placeholder", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column1); + OlapTable table1 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + OlapTable table2 = new OlapTable(id++, "testTable" + id + "_1", schema, null, null, null); + db.createTableWithLock(table1, true, false); + db.createTableWithLock(table2, true, false); + } + + testLowMap.clear(); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); + appender.appendToLowJobs(testLowMap, testVeryLowMap); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); + appender.appendToLowJobs(testLowMap, testVeryLowMap); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - Assertions.assertEquals(StatisticsJobAppender.JOB_MAP_SIZE, testMap.size()); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(0, testLowMap.size()); + Assertions.assertEquals(StatisticsJobAppender.JOB_MAP_SIZE, testVeryLowMap.size()); } @Test @@ -270,12 +335,48 @@ public int getAutoAnalyzeTableWidthThreshold() { return thresholds[count++]; } }; - Map>> testMap = new HashMap<>(); + Map>> testLowMap = new HashMap<>(); + Map>> testVeryLowMap = new HashMap<>(); StatisticsJobAppender appender = new StatisticsJobAppender(); - appender.appendToLowJobs(testMap); - Assertions.assertEquals(0, testMap.size()); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(0, testLowMap.size()); appender.setLastRoundFinishTime(0); - appender.appendToLowJobs(testMap); - Assertions.assertEquals(1, testMap.size()); + appender.appendToLowJobs(testLowMap, testVeryLowMap); + Assertions.assertEquals(1, testLowMap.size()); + } + + @Test + public void testDoAppend() { + Map>> jobMap = Maps.newHashMap(); + Set> columnIndexPairs1 = Sets.newHashSet(); + Set> columnIndexPairs2 = Sets.newHashSet(); + TableName tableName1 = new TableName("catalog1", "db1", "table1"); + TableName tableName2 = new TableName("catalog2", "db2", "table2"); + Pair pair1 = Pair.of("index1", "col1"); + columnIndexPairs1.add(pair1); + + StatisticsJobAppender appender = new StatisticsJobAppender(); + Assertions.assertTrue(appender.doAppend(jobMap, columnIndexPairs1, tableName1)); + Assertions.assertEquals(1, jobMap.size()); + Assertions.assertTrue(jobMap.containsKey(tableName1)); + Assertions.assertEquals(1, jobMap.get(tableName1).size()); + Assertions.assertTrue(jobMap.get(tableName1).contains(pair1)); + + Pair pair2 = Pair.of("index2", "col2"); + columnIndexPairs1.add(pair2); + Assertions.assertTrue(appender.doAppend(jobMap, columnIndexPairs1, tableName1)); + Assertions.assertEquals(1, jobMap.size()); + Assertions.assertTrue(jobMap.containsKey(tableName1)); + Assertions.assertEquals(2, jobMap.get(tableName1).size()); + Assertions.assertTrue(jobMap.get(tableName1).contains(pair1)); + Assertions.assertTrue(jobMap.get(tableName1).contains(pair2)); + + Pair pair3 = Pair.of("index3", "col3"); + columnIndexPairs2.add(pair3); + Assertions.assertTrue(appender.doAppend(jobMap, columnIndexPairs2, tableName2)); + Assertions.assertEquals(2, jobMap.size()); + Assertions.assertTrue(jobMap.containsKey(tableName2)); + Assertions.assertEquals(1, jobMap.get(tableName2).size()); + Assertions.assertTrue(jobMap.get(tableName2).contains(pair3)); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java index fbac718e421c29..ef1e9ca02970b1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/statistics/util/StatisticsUtilTest.java @@ -32,6 +32,7 @@ import org.apache.doris.datasource.hive.HMSExternalCatalog; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSExternalTable.DLAType; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.jdbc.JdbcExternalCatalog; import org.apache.doris.datasource.jdbc.JdbcExternalTable; import org.apache.doris.qe.SessionVariable; @@ -208,12 +209,6 @@ public TableStatsMeta findTableStatsStatus(long tblId) { Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); // Test external table auto analyze enabled. - new MockUp() { - @Mock - public TableStatsMeta findTableStatsStatus(long tblId) { - return null; - } - }; externalCatalog.getCatalogProperty().addProperty(ExternalCatalog.ENABLE_AUTO_ANALYZE, "false"); HMSExternalTable hmsTable1 = new HMSExternalTable(1, "name", "dbName", externalCatalog); externalCatalog.setAutoAnalyzePolicy("dbName", "name", "enable"); @@ -238,27 +233,10 @@ public TableStatsMeta findTableStatsStatus(long tblId) { tableMeta.userInjected = false; Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); - // Test column hasn't been analyzed for longer than 1 day. new MockUp() { @Mock public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { - return new ColStatsMeta(0, null, null, null, 0, 100, 0, null); - } - }; - new MockUp() { - @Mock - public long getRowCount() { - return 100; - } - }; - Config.auto_analyze_interval_seconds = 60 * 60 * 24; - Assertions.assertTrue(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); - Config.auto_analyze_interval_seconds = 0; - - new MockUp() { - @Mock - public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { - return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 0, 0, null); + return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 0, 0, 0, null); } }; @@ -312,7 +290,7 @@ public long getRowCount() { new MockUp() { @Mock public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { - return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 100, 0, null); + return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 100, 0, 0, null); } }; tableMeta.partitionChanged.set(false); @@ -322,7 +300,7 @@ public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { new MockUp() { @Mock public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { - return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 0, 0, null); + return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 0, 0, 0, null); } }; tableMeta.partitionChanged.set(false); @@ -338,7 +316,7 @@ public long getRowCount() { new MockUp() { @Mock public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { - return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 500, 0, null); + return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 500, 0, 0, null); } }; tableMeta.partitionChanged.set(false); @@ -354,7 +332,7 @@ public long getRowCount() { new MockUp() { @Mock public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { - return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 100, 80, null); + return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 100, 80, 0, null); } }; tableMeta.partitionChanged.set(false); @@ -382,6 +360,140 @@ public long getRowCount() { tableMeta.partitionChanged.set(false); tableMeta.updatedRows.set(85); Assertions.assertFalse(StatisticsUtil.needAnalyzeColumn(table, Pair.of("index", column.getName()))); + } + + @Test + void testLongTimeNoAnalyze() { + Column column = new Column("testColumn", PrimitiveType.INT); + List schema = new ArrayList<>(); + schema.add(column); + OlapTable table = new OlapTable(200, "testTable", schema, null, null, null); + // Test column is null + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, null)); + + // Test table auto analyze is disabled. + new MockUp() { + @Mock + public boolean autoAnalyzeEnabled() { + return false; + } + }; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + new MockUp() { + @Mock + public boolean autoAnalyzeEnabled() { + return true; + } + }; + + // Test external table + new MockUp() { + @Mock + public boolean autoAnalyzeEnabled() { + return true; + } + }; + IcebergExternalTable icebergTable = new IcebergExternalTable(0, "", "", null); + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(icebergTable, Pair.of("index", column.getName()))); + + // Test table stats meta is null. + new MockUp() { + @Mock + public TableStatsMeta findTableStatsStatus(long tblId) { + return null; + } + }; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + + // Test column stats meta is null + TableStatsMeta tableMeta = new TableStatsMeta(); + new MockUp() { + @Mock + public TableStatsMeta findTableStatsStatus(long tblId) { + return tableMeta; + } + }; + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return null; + } + }; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + return new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 100, 0, 0, null); + } + }; + + // Test table stats is user injected + tableMeta.userInjected = true; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + tableMeta.userInjected = false; + + // Test Config.auto_analyze_interval_seconds == 0 + Config.auto_analyze_interval_seconds = 0; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + + // Test column analyzed within the time interval + Config.auto_analyze_interval_seconds = 86400; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + + // Test column hasn't analyzed for longer than time interval, but version and row count doesn't change + new MockUp() { + @Mock + public ColStatsMeta findColumnStatsMeta(String indexName, String colName) { + ColStatsMeta ret = new ColStatsMeta(System.currentTimeMillis(), null, null, null, 0, 100, 20, 10, null); + try { + Thread.sleep(1500); + } catch (InterruptedException e) { + e.printStackTrace(); + } + return ret; + } + }; + new MockUp() { + @Mock + public long getVisibleVersion() { + return 10; + } + + @Mock + public long fetchRowCount() { + return 100; + } + }; + Config.auto_analyze_interval_seconds = 1; + Assertions.assertFalse(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + + // Test column hasn't analyzed for longer than time interval, and version change + new MockUp() { + @Mock + public long getVisibleVersion() { + return 11; + } + + @Mock + public long fetchRowCount() { + return 100; + } + }; + Assertions.assertTrue(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); + + // Test column hasn't analyzed for longer than time interval, and row count change + new MockUp() { + @Mock + public long getVisibleVersion() { + return 10; + } + + @Mock + public long fetchRowCount() { + return 101; + } + }; + Assertions.assertTrue(StatisticsUtil.isLongTimeColumn(table, Pair.of("index", column.getName()))); } } diff --git a/fe/pom.xml b/fe/pom.xml index d78cfd50b819b4..0d2e3f70aa6e6a 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -317,7 +317,7 @@ under the License. 1.11.4 17.0.0 - 0.14.1 + 0.15.0 2.7.4-11 3.0.0-8 @@ -371,7 +371,7 @@ under the License. 435 2.1.1 9.4 - 202 + 202 1.2.27 12.22.0 5.3.0 @@ -1649,7 +1649,7 @@ under the License. io.airlift concurrent - ${airlift.version} + ${airlift.concurrent.version} com.azure diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 1c217a9d36ba86..a18f7a262973d1 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -53,6 +53,7 @@ struct TColumnDesc { 6: optional bool isAllowNull 7: optional string columnKey 8: optional list children + 9: optional string defaultValue } // A column definition; used by CREATE TABLE and DESCRIBE

statements. A column diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 0bbd364fda1c2a..62e88621aeb769 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -353,7 +353,10 @@ struct TMaxComputeFileDesc { 1: optional string partition_spec // deprecated 2: optional string session_id 3: optional string table_batch_read_session - + // for mc network configuration + 4: optional i32 connect_timeout + 5: optional i32 read_timeout + 6: optional i32 retry_times } struct THudiFileDesc { @@ -367,6 +370,7 @@ struct THudiFileDesc { 8: optional list column_names; 9: optional list column_types; 10: optional list nested_fields; + 11: optional string hudi_jni_scanner; } struct TLakeSoulFileDesc { @@ -406,6 +410,7 @@ enum TTextSerdeType { struct TFileScanRangeParams { // deprecated, move to TFileScanRange 1: optional Types.TFileType file_type; + // deprecated, move to TFileScanRange 2: optional TFileFormatType format_type; // deprecated, move to TFileScanRange 3: optional TFileCompressType compress_type; @@ -480,6 +485,7 @@ struct TFileRangeDesc { // for hive table, different files may have different fs, // so fs_name should be with TFileRangeDesc 12: optional string fs_name + 13: optional TFileFormatType format_type; } struct TSplitSource { diff --git a/regression-test/data/account_p0/test_information_schema.out b/regression-test/data/account_p0/test_information_schema.out index 77d5f6dccd50e8..e4f0ed09d0e1c7 100644 --- a/regression-test/data/account_p0/test_information_schema.out +++ b/regression-test/data/account_p0/test_information_schema.out @@ -17,3 +17,10 @@ -- !sql -- DUP +-- !default -- +id largeint YES \N +name varchar(20) YES 无 +age smallint(6) YES 0 +address varchar(100) YES beijing +date datetime YES 20240101 + diff --git a/regression-test/data/correctness_p0/test_mv_case/test_mv_case.out b/regression-test/data/correctness_p0/test_mv_case/test_mv_case.out index e87ee694dc1adb..bab672bd6abd07 100644 --- a/regression-test/data/correctness_p0/test_mv_case/test_mv_case.out +++ b/regression-test/data/correctness_p0/test_mv_case/test_mv_case.out @@ -5,3 +5,6 @@ cib2205045_1_1s 2023-06-10T03:56:33 {"DB1":168939,"DNT":"2023-06-10 03:56:33"} cib2205045_1_1s 2023-06-10T03:57:33 {"DB1":168939,"DNT":"2023-06-10 03:57:33"} cib2205045_1_1s 2023-06-10T03:58:33 {"DB1":168939,"DNT":"2023-06-10 03:58:33"} +-- !select_mv -- +123 {3:"5", 6:"8"} + diff --git a/regression-test/data/export_p0/outfile/test_outfile_complex_type.out b/regression-test/data/export_p0/outfile/test_outfile_complex_type.out new file mode 100644 index 00000000000000..cd6f000b6c5052 --- /dev/null +++ b/regression-test/data/export_p0/outfile/test_outfile_complex_type.out @@ -0,0 +1,25 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_load_parquet -- +20220201 0 0000004501000000000000F03F 0101675D86AC33FA8CD6 +20220201 1 0000004501000000000000F0BF 01010B3C52B765A11A2F +20220201 2 00000045010000000000000000 0101DDEA60F9C89AA329 +20220201 3 0000004501000000000000F03F 0101EF81F59130F8B748 +20220201 4 00000045010000000000000040 010114CAA737BD54146E +20220201 5 00000045010000000000000840 0101DCBC5BA258F9602C + +-- !select_load_orc -- +20220201 0 0000004501000000000000F03F 0101675D86AC33FA8CD6 +20220201 1 0000004501000000000000F0BF 01010B3C52B765A11A2F +20220201 2 00000045010000000000000000 0101DDEA60F9C89AA329 +20220201 3 0000004501000000000000F03F 0101EF81F59130F8B748 +20220201 4 00000045010000000000000040 010114CAA737BD54146E +20220201 5 00000045010000000000000840 0101DCBC5BA258F9602C + +-- !select_load_csv -- +20220201 0 \N \N \N +20220201 1 \N \N \N +20220201 2 \N \N \N +20220201 3 \N \N \N +20220201 4 \N \N \N +20220201 5 \N \N \N + diff --git a/regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out b/regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out new file mode 100644 index 00000000000000..d25830939645f6 --- /dev/null +++ b/regression-test/data/export_p0/outfile/test_outfile_jsonb_and_variant.out @@ -0,0 +1,25 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_load_parquet -- +20220201 0 {"k1":"100"} {"k1":"100"} +20220201 1 {"k1":"100","k2":"123"} {"k1":"100","k2":"123"} +20220201 2 {"k1":"100","abc":"567"} {"abc":"567","k1":"100"} +20220201 3 {"k1":"100","k3":123} {"k1":"100","k3":123} +20220201 4 {"k1":"100","doris":"nereids"} {"doris":"nereids","k1":"100"} +20220201 5 {"k1":"100","doris":"pipeline"} {"doris":"pipeline","k1":"100"} + +-- !select_load_orc -- +20220201 0 {"k1":"100"} {"k1":"100"} +20220201 1 {"k1":"100","k2":"123"} {"k1":"100","k2":"123"} +20220201 2 {"k1":"100","abc":"567"} {"abc":"567","k1":"100"} +20220201 3 {"k1":"100","k3":123} {"k1":"100","k3":123} +20220201 4 {"k1":"100","doris":"nereids"} {"doris":"nereids","k1":"100"} +20220201 5 {"k1":"100","doris":"pipeline"} {"doris":"pipeline","k1":"100"} + +-- !select_load_orc -- +20220201 0 {"k1":"100"} {"k1":"100"} +20220201 1 {"k1":"100","k2":"123"} {"k1":"100","k2":"123"} +20220201 2 {"k1":"100","abc":"567"} {"abc":"567","k1":"100"} +20220201 3 {"k1":"100","k3":123} {"k1":"100","k3":123} +20220201 4 {"k1":"100","doris":"nereids"} {"doris":"nereids","k1":"100"} +20220201 5 {"k1":"100","doris":"pipeline"} {"doris":"pipeline","k1":"100"} + diff --git a/regression-test/data/export_p0/outfile/test_outfile_null_type.out b/regression-test/data/export_p0/outfile/test_outfile_null_type.out new file mode 100644 index 00000000000000..1777bba72c1b5c --- /dev/null +++ b/regression-test/data/export_p0/outfile/test_outfile_null_type.out @@ -0,0 +1,37 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_export -- +1 ftw-1 19 +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 +10 \N \N + +-- !select_load_parquet -- +1 ftw-1 19 \N +10 \N \N \N +2 ftw-2 20 \N +3 ftw-3 21 \N +4 ftw-4 22 \N +5 ftw-5 23 \N +6 ftw-6 24 \N +7 ftw-7 25 \N +8 ftw-8 26 \N +9 ftw-9 27 \N + +-- !select_load_csv -- +1 ftw-1 19 \N +10 \N \N \N +2 ftw-2 20 \N +3 ftw-3 21 \N +4 ftw-4 22 \N +5 ftw-5 23 \N +6 ftw-6 24 \N +7 ftw-7 25 \N +8 ftw-8 26 \N +9 ftw-9 27 \N + diff --git a/regression-test/data/external_table_p0/hive/hive_json_basic_test.out b/regression-test/data/external_table_p0/hive/hive_json_basic_test.out new file mode 100644 index 00000000000000..9023f5d72b1ac3 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/hive_json_basic_test.out @@ -0,0 +1,115 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 +4 \N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N dt2 +5 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 +6 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 + +-- !q2 -- +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 + +-- !q3 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 + +-- !q4 -- +123.45 +789.56 +-3.4028235e+38 +123.45 + +-- !q5 -- +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 + +-- !q6 -- +user1 {"name":"Alice", "age":28, "preferences":{"sports":{"preference_id":101, "preference_values":["soccer", "tennis"]}, "music":{"preference_id":102, "preference_values":["rock", "classical"]}}} [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 {"name":"Bob", "age":32, "preferences":{"books":{"preference_id":201, "preference_values":["fiction", "non-fiction"]}, "travel":{"preference_id":202, "preference_values":["beaches", "mountains"]}}} [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q7 -- +user1 [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q8 -- +\N \N \N \N +\N \N \N \N +1 10 {"col2a":10, "col2b":"string1"} {1:"string10"} +2 20 {"col2a":20, "col2b":"string2"} {2:"string2"} +3 30 {"col2a":30, "col2b":"string3"} \N +4 40 {"col2a":40, "col2b":"string4"} {4:"string4"} +5 \N \N \N +6 60 {"col2a":60, "col2b":"600"} {6:"600"} +7 70 {"col2a":70, "col2b":"string7"} {7:"string7"} + +-- !q9 -- +\N \N +\N \N +\N 5 +10 1 +20 2 +30 3 +40 4 +60 6 +70 7 + +-- !q1 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 +4 \N \N \N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N dt2 +5 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 +6 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N dt3 + +-- !q2 -- +3 false -128 -32768 -2147483648 -9223372036854775808 -3.4028235e+38 -1.7976931348623157E308 -999999999 -9999.9999 -999999999.999999 \N min_string_value xxxx 2001-01-01 2001-01-01T00:00 2001-01-01T00:00 2001-01-01T00:00 char_min_value1 char_min_value2 char_min_value3 varchar_min_value1 varchar_min_value2 varchar_min_value3 {"min_key":"min_value"} {"min_key":"min_value"} {"a ":"z "} {-1:-100} {-1:-100000000000} {-1.1:-10.1} {-1.1:-10.1} {0:1} {-1.1:-1.1} {-1.23:-1.23} {-1.2345:-1.2345} {-1.23456789:-1.23456789} {-1.23456789:-1.23456789} {-1.2345678901234568:-1.2345678901234568} ["min_string1", "min_string2"] [-10, -5, -3] [-100000000000, -200000000000] [-1.1, -2.2] [-1.123456789, -2.123456789] [0, 1] ["min_varchar1", "min_varchar2"] ["min_char1 ", "min_char2 "] [-1.1, -2.2] [-1.23, -2.34] [-1.2345, -2.3456] [-1.23456789, -2.34567891] [-1.23456789, -2.34567891] [-1.2345678901234568, -2.3456789012345679] {"s_bigint":-1234567890} {"min_key":[{"s_int":-123}]} {"struct_field":["min_value1", "min_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"min_value"} {"struct_field1":-123, "struct_field2":"min_value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "min_value1", "min_value2"] ["min_value1", null, "min_value2"] ["min_value1", "min_value2", null] [null, null, null] dt1 + +-- !q3 -- +1 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value binary_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a ":"b "} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1 ", "char2 "] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} {"null_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] dt1 + +-- !q4 -- +123.45 +789.56 +-3.4028235e+38 +123.45 + +-- !q5 -- +2 false 58 12345 2147483000 \N 789.56 654321.123 987654321 5678.1234 987654.321098 987654321.098765432109 changed_string new_binary_value 2025-05-25 2025-05-25T15:30 2025-05-25T15:30:00.654322 2025-05-25T15:30:00.654322 char_new_value1 char_new_value2 char_new_value3 varchar_new_value1 varchar_new_value2 varchar_new_value3 {"key2":"value2"} {"key2":"value2"} {"x ":"y "} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567891:2.34567891} {2.34567891:2.34567891} {2.3456789012345679:2.3456789012345679} ["string3", "string4"] [4, 5, 6] [300000000000, 400000000000] [2.2, 3.3] [2.123456789, 3.123456789] [0, 1] ["varchar3", "varchar4"] ["char3 ", "char4 "] [2.2, 3.3] [2.34, 3.45] [2.3456, 3.4567] [2.34567891, 3.45678901] [2.34567891, 3.45678901] [2.3456789012345679, 3.4567890123456789] {"s_bigint":9876543210} {"key2":[{"s_int":456}]} {"struct_field":["new_value1", "new_value2"]} {"struct_field_null":null, "struct_field_null2":null} {"struct_non_nulls_after_nulls1":456, "struct_non_nulls_after_nulls2":"new_value"} {"struct_field1":456, "struct_field2":"new_value", "strict_field3":{"nested_struct_field1":456, "nested_struct_field2":"nested_value2"}} {"null_key":null} [null, "new_value1", "new_value2"] ["new_value1", null, "new_value2"] ["new_value1", "new_value2", null] [null, null, null] dt1 + +-- !q6 -- +user1 {"name":"Alice", "age":28, "preferences":{"sports":{"preference_id":101, "preference_values":["soccer", "tennis"]}, "music":{"preference_id":102, "preference_values":["rock", "classical"]}}} [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 {"name":"Bob", "age":32, "preferences":{"books":{"preference_id":201, "preference_values":["fiction", "non-fiction"]}, "travel":{"preference_id":202, "preference_values":["beaches", "mountains"]}}} [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 {"name":"Carol", "age":24, "preferences":{"food":{"preference_id":301, "preference_values":["vegan", "desserts"]}, "movies":{"preference_id":302, "preference_values":["action", "comedy"]}}} [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q7 -- +user1 [{"activity_date":"2024-08-01", "activities":{"workout":{"details":"Morning run", "metrics":{"duration":30.5, "calories":200}}, "reading":{"details":"Read book on Hive", "metrics":{"pages":50, "time":2}}}}, {"activity_date":"2024-08-02", "activities":{"travel":{"details":"Flight to NY", "metrics":{"distance":500, "time":3}}, "meeting":{"details":"Project meeting", "metrics":{"duration":1.5, "participants":5}}}}] +user2 [{"activity_date":"2024-08-01", "activities":{"hiking":{"details":"Mountain trail", "metrics":{"distance":10, "elevation":500}}, "photography":{"details":"Wildlife photoshoot", "metrics":{"photos_taken":100, "time":4}}}}, {"activity_date":"2024-08-02", "activities":{"workshop":{"details":"Photography workshop", "metrics":{"duration":3, "participants":15}}, "shopping":{"details":"Bought camera gear", "metrics":{"items":5, "cost":1500}}}}] +user3 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] +user4 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"duration":2, "rating":8.5}}}}, {"activity_date":"2024-08-02", "activities":{"gym":{"details":"Strength training", "metrics":{"duration":1, "calories":300}}, "shopping":{"details":"Bought groceries", "metrics":{"items":10, "cost":100}}}}] + +-- !q8 -- +\N \N \N \N +\N \N \N \N +1 10 {"col2a":10, "col2b":"string1"} {1:"string10"} +2 20 {"col2a":20, "col2b":"string2"} {2:"string2"} +3 30 {"col2a":30, "col2b":"string3"} \N +4 40 {"col2a":40, "col2b":"string4"} {4:"string4"} +5 \N \N \N +6 60 {"col2a":60, "col2b":"600"} {6:"600"} +7 70 {"col2a":70, "col2b":"string7"} {7:"string7"} + +-- !q9 -- +\N \N +\N \N +\N 5 +10 1 +20 2 +30 3 +40 4 +60 6 +70 7 + diff --git a/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out b/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out new file mode 100644 index 00000000000000..42414c3654930c --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/iceberg_read_unitycatalog_table.out @@ -0,0 +1,40 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1 -- +1 nWYHawtqUw 930 +2 uvOzzthsLV 166 +3 WIAehuXWkv 170 +4 wYCSvnJKTo 709 +5 VsslXsUIDZ 993 +6 ZLsACYYTFy 813 +7 BtDDvLeBpK 52 +8 YISVtrPfGr 8 +9 PBPJHDFjjC 45 +10 qbDuUJzJMO 756 +11 EjqqWoaLJn 712 +12 jpZLMdKXpn 847 +13 acpjQXpJCp 649 +14 nOKqHhRwao 133 +15 kxUUZEUoKv 398 + +-- !q2 -- +7 +8 +9 +10 +11 +12 +13 +14 +15 + +-- !q3 -- +nWYHawtqUw 930 +wYCSvnJKTo 709 +VsslXsUIDZ 993 +ZLsACYYTFy 813 +qbDuUJzJMO 756 +EjqqWoaLJn 712 +jpZLMdKXpn 847 +acpjQXpJCp 649 +kxUUZEUoKv 398 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out b/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out index b1bdad85013bfc..50644f34961942 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_incremental.out @@ -347,3 +347,177 @@ -- !incremental_9_10 -- 1000 +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + +-- !incremental_1_end -- +9000 + +-- !incremental_earliest_1 -- +1000 + +-- !incremental_2_end -- +8000 + +-- !incremental_earliest_2 -- +2000 + +-- !incremental_1_2 -- +1000 + +-- !incremental_3_end -- +7000 + +-- !incremental_earliest_3 -- +3000 + +-- !incremental_2_3 -- +1000 + +-- !incremental_4_end -- +6000 + +-- !incremental_earliest_4 -- +4000 + +-- !incremental_3_4 -- +1000 + +-- !incremental_5_end -- +5000 + +-- !incremental_earliest_5 -- +5000 + +-- !incremental_4_5 -- +1000 + +-- !incremental_6_end -- +4000 + +-- !incremental_earliest_6 -- +6000 + +-- !incremental_5_6 -- +1000 + +-- !incremental_7_end -- +3000 + +-- !incremental_earliest_7 -- +7000 + +-- !incremental_6_7 -- +1000 + +-- !incremental_8_end -- +2000 + +-- !incremental_earliest_8 -- +8000 + +-- !incremental_7_8 -- +1000 + +-- !incremental_9_end -- +1000 + +-- !incremental_earliest_9 -- +9000 + +-- !incremental_8_9 -- +1000 + +-- !incremental_10_end -- +0 + +-- !incremental_earliest_10 -- +10000 + +-- !incremental_9_10 -- +1000 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_orc_tables.out b/regression-test/data/external_table_p2/hudi/test_hudi_orc_tables.out new file mode 100644 index 00000000000000..9e28074dc9114e --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_orc_tables.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cow -- +20241204190011744 20241204190011744_0_6 20241204190011744_0_0 a99e363a-6c10-40f3-a675-9117506d1a43-0_0-38-94_20241204190011744.orc 1 A +20241204190011744 20241204190011744_0_7 20241204190011744_2_0 a99e363a-6c10-40f3-a675-9117506d1a43-0_0-38-94_20241204190011744.orc 3 C +20241204190011744 20241204190011744_0_8 20241204190011744_4_0 a99e363a-6c10-40f3-a675-9117506d1a43-0_0-38-94_20241204190011744.orc 5 E +20241204190011744 20241204190011744_0_9 20241204190011744_1_0 a99e363a-6c10-40f3-a675-9117506d1a43-0_0-38-94_20241204190011744.orc 2 B +20241204190011744 20241204190011744_0_10 20241204190011744_3_0 a99e363a-6c10-40f3-a675-9117506d1a43-0_0-38-94_20241204190011744.orc 4 D + +-- !mor -- +20241204190002046 20241204190002046_0_11 20241204190002046_0_0 b1e68412-01d6-467f-b4c2-b4b18ec71346-0_0-30-75_20241204190002046.orc 1 A +20241204190002046 20241204190002046_0_12 20241204190002046_2_0 b1e68412-01d6-467f-b4c2-b4b18ec71346-0_0-30-75_20241204190002046.orc 3 C +20241204190002046 20241204190002046_0_13 20241204190002046_4_0 b1e68412-01d6-467f-b4c2-b4b18ec71346-0_0-30-75_20241204190002046.orc 5 E +20241204190002046 20241204190002046_0_14 20241204190002046_1_0 b1e68412-01d6-467f-b4c2-b4b18ec71346-0_0-30-75_20241204190002046.orc 2 B +20241204190002046 20241204190002046_0_15 20241204190002046_3_0 b1e68412-01d6-467f-b4c2-b4b18ec71346-0_0-30-75_20241204190002046.orc 4 D + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_partition_prune.out b/regression-test/data/external_table_p2/hudi/test_hudi_partition_prune.out new file mode 100644 index 00000000000000..fd3eafa0255722 --- /dev/null +++ b/regression-test/data/external_table_p2/hudi/test_hudi_partition_prune.out @@ -0,0 +1,357 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !time_travel_two_partition_1_3 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 + +-- !time_travel_two_partition_2_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !time_travel_two_partition_3_1 -- +4 David US 2 +5 Eva US 2 + +-- !time_travel_two_partition_4_0 -- + +-- !time_travel_two_partition_5_0 -- + +-- !time_travel_two_partition_6_1 -- +1 Alice US 1 + +-- !one_partition_boolean -- +1 Alice true +2 Bob true + +-- !one_partition_tinyint -- +1 Alice 1 +2 Bob 1 + +-- !one_partition_smallint -- +1 Alice 10 +2 Bob 10 + +-- !one_partition_int -- +1 Alice 100 +2 Bob 100 + +-- !one_partition_bigint -- +1 Alice 1234567890 +2 Bob 1234567890 + +-- !one_partition_string -- +1 Alice RegionA +2 Bob RegionA + +-- !one_partition_date -- +1 Alice 2023-12-01 +2 Bob 2023-12-01 + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !time_travel_two_partition_1_3 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 + +-- !time_travel_two_partition_2_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !time_travel_two_partition_3_1 -- +4 David US 2 +5 Eva US 2 + +-- !time_travel_two_partition_4_0 -- + +-- !time_travel_two_partition_5_0 -- + +-- !time_travel_two_partition_6_1 -- +1 Alice US 1 + +-- !one_partition_boolean -- +1 Alice true +2 Bob true + +-- !one_partition_tinyint -- +1 Alice 1 +2 Bob 1 + +-- !one_partition_smallint -- +1 Alice 10 +2 Bob 10 + +-- !one_partition_int -- +1 Alice 100 +2 Bob 100 + +-- !one_partition_bigint -- +1 Alice 1234567890 +2 Bob 1234567890 + +-- !one_partition_string -- +1 Alice RegionA +2 Bob RegionA + +-- !one_partition_date -- +1 Alice 2023-12-01 +2 Bob 2023-12-01 + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out b/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out index 12dd0cf086d3f0..da7273d4c14ef9 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_schema_evolution.out @@ -31,3 +31,35 @@ 20241118012149007 20241118012149007_0_4 5 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 5 Eva {"age":31.5, "address":"Chengdu"} 20241118012149007 20241118012149007_0_5 6 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 6 Frank {"age":29.2, "address":"Wuhan"} +-- !adding_simple_columns_table -- +20241118012126237 20241118012126237_0_1 1 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 1 Alice \N +20241118012126237 20241118012126237_0_0 2 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 2 Bob \N +20241118012126237 20241118012126237_0_2 3 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 3 Cathy \N +20241118012132306 20241118012132306_0_3 4 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 4 David 25 +20241118012132306 20241118012132306_0_4 5 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 5 Eva 30 +20241118012132306 20241118012132306_0_5 6 5166112a-90d8-4ba8-8646-337fbeb2a375-0_0-35-121_20241118012132306.parquet 6 Frank 28 + +-- !altering_simple_columns_table -- +20241118012136512 20241118012136512_0_0 1 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 1 Alice 25.0 +20241118012136512 20241118012136512_0_2 2 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 2 Bob 30.0 +20241118012136512 20241118012136512_0_1 3 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 3 Cathy 28.0 +20241118012138287 20241118012138287_0_3 4 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 4 David 26.0 +20241118012138287 20241118012138287_0_4 5 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 5 Eva 31.5 +20241118012138287 20241118012138287_0_5 6 203f0f43-ae9d-4c17-8d5d-834f0dbc62c9-0_0-78-246_20241118012138287.parquet 6 Frank 29.2 + +-- !adding_complex_columns_table -- +20241118012144831 20241118012144831_0_1 1 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 1 Alice {"age":25, "address":"Guangzhou", "email":null} +20241118012144831 20241118012144831_0_0 2 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 2 Bob {"age":30, "address":"Shanghai", "email":null} +20241118012144831 20241118012144831_0_2 3 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 3 Cathy {"age":28, "address":"Beijing", "email":null} +20241118012146150 20241118012146150_0_3 4 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 4 David {"age":25, "address":"Shenzhen", "email":"david@example.com"} +20241118012146150 20241118012146150_0_4 5 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 5 Eva {"age":30, "address":"Chengdu", "email":"eva@example.com"} +20241118012146150 20241118012146150_0_5 6 3c038df9-a652-4878-9b8a-221ae443448e-0_0-165-497_20241118012146150.parquet 6 Frank {"age":28, "address":"Wuhan", "email":"frank@example.com"} + +-- !altering_complex_columns_table -- +20241118012147879 20241118012147879_0_0 1 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 1 Alice {"age":25, "address":"Guangzhou"} +20241118012147879 20241118012147879_0_2 2 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 2 Bob {"age":30, "address":"Shanghai"} +20241118012147879 20241118012147879_0_1 3 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 3 Cathy {"age":28, "address":"Beijing"} +20241118012149007 20241118012149007_0_3 4 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 4 David {"age":26, "address":"Shenzhen"} +20241118012149007 20241118012149007_0_4 5 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 5 Eva {"age":31.5, "address":"Chengdu"} +20241118012149007 20241118012149007_0_5 6 185d101f-a484-45ce-b236-03ccd33c521b-0_0-208-622_20241118012149007.parquet 6 Frank {"age":29.2, "address":"Wuhan"} + diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out b/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out index efad67ffbfa8c4..1e151c2a86fa20 100644 Binary files a/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out and b/regression-test/data/external_table_p2/hudi/test_hudi_snapshot.out differ diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out b/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out index dc47ff86d90a8d..9bdb0f7cb7285f 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_timestamp.out @@ -1,6 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !timestamp -- +-- !timestamp1 -- 20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T08:00 -20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T09:30:00 -20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T11:00:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T09:30 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T11:00 + +-- !timestamp2 -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T23:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-26T00:30 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-26T02:00 + +-- !timestamp3 -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T15:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T16:30 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T18:00 + +-- !timestamp1 -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T08:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T09:30 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T11:00 + +-- !timestamp2 -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T23:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-26T00:30 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-26T02:00 + +-- !timestamp3 -- +20241115015956800 20241115015956800_0_2 1 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 1 Alice 2024-10-25T15:00 +20241115015956800 20241115015956800_0_0 2 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 2 Bob 2024-10-25T16:30 +20241115015956800 20241115015956800_0_1 3 eec4913a-0d5f-4b8b-a0f5-934e252c2e45-0_0-7-14_20241115015956800.parquet 3 Charlie 2024-10-25T18:00 diff --git a/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out b/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out index a9b5d23595a8e3..00d15805baf04e 100644 --- a/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out +++ b/regression-test/data/external_table_p2/hudi/test_hudi_timetravel.out @@ -119,3 +119,123 @@ -- !timetravel10 -- 10000 +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + +-- !timetravel1 -- +1000 + +-- !timetravel2 -- +2000 + +-- !timetravel3 -- +3000 + +-- !timetravel4 -- +4000 + +-- !timetravel5 -- +5000 + +-- !timetravel6 -- +6000 + +-- !timetravel7 -- +7000 + +-- !timetravel8 -- +8000 + +-- !timetravel9 -- +9000 + +-- !timetravel10 -- +10000 + diff --git a/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out new file mode 100644 index 00000000000000..d6c23e6b5abf52 --- /dev/null +++ b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out @@ -0,0 +1,1489 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + diff --git a/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out b/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out index 1da845620e8dd7..c00bbc9377409e 100644 --- a/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out +++ b/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out @@ -6,10 +6,13 @@ 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#1 category brand color type 4 container 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#1 category brand color type 4 container 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 +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 +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 +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 -- !select_mv -- -4 +16 -- !select -- -4 +16 diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query21.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query21.out index 2076a345643b22..50515df29069f2 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query21.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query21.out @@ -19,7 +19,7 @@ PhysicalResultSink ------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------------PhysicalOlapScan[item] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-07-22') and (date_dim.d_date >= '1999-05-23')) +--------------------------filter((date_dim.d_date = '1999-05-23')) ----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out index b8219c68f1fe8b..04e0789bf8a35e 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out @@ -23,7 +23,7 @@ PhysicalResultSink --------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ----------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +----------------------------filter((date_dim.d_date = '2001-04-02')) ------------------------------PhysicalOlapScan[date_dim] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_p0/sql_functions/array_functions/array_filter.csv b/regression-test/data/nereids_p0/sql_functions/array_functions/array_filter.csv new file mode 100644 index 00000000000000..ebb82b870b62cf --- /dev/null +++ b/regression-test/data/nereids_p0/sql_functions/array_functions/array_filter.csv @@ -0,0 +1,236 @@ +1 2147483647 0 2147483647 -774774309 2024-02-18 2014-08-12 2024-02-18 2023-12-15 几次 h who tell something mean there do want s 高级总代理 办款千兆这点彭伟按季度是你或 -- 0 1 \N 40.123000 1.200000 100.020000 2024-01-09 00:00:00 2023-12-10 00:00:00 2023-12-12 00:00:00 2023-01-15 08:32:59.123000 2023-12-09 00:00:00 2024-01-19 00:00:00 2024-01-19 00:00:00 2024-01-19 00:00:00 [1, 1, 1] [] [1, 1, 1, 0, 0, 1, 1, 1] [1, 0, 1] \N [127, -1, 0, 2, 0, -40, 0, -1] [1, 0, 127, 80, 4, 2, -1, 0] [-128, -77, 0, 127, -109, 127, 127, 0, 127, 0] [32767, -25368, 0] [1, 1, 13043, 7678, 32767] [-659, 245, -32622, -31408, -32768, 0, 26805, -11899, 1, -8576] [1, -32768, 9173] \N [0, -1, -739795943, 32679, -653641418, 1, 147483648, -1, 1658834108, 75627634] [-1405547186, 612389568, -7871595] [] [9223372036854775807, 1, -1, 1, 147483648, -247522161142036907, -1718099495, -7275653850596166809] [1, -1, -2325175214315801374, 5609434693957453270, 1, 8170474, -7266240, -1] [-8417662042588257736, 332023890725204128, 1819780, 1, 1574074550688853324, -208716795, -1, 1] [-9223372036854775808, -6540400, 44377, 32679, -1, 9223372036854775807, 1363806540, -585226604] [2061954133, -2, 1, 0, 1, 0, 0, 1, 8, -1] [9, -2, 0] [3, 1, 0, 1332681777, 0, -2, -1, 9, 1, 0] [0, 1, 0] [14.050200000000, 98.194600000000, 42.095800000000, 100.020000000000, 0.000000000000] [300.343000000000, 0.000000000000, 40.123000000000, 40.123000000000, 300.343000000000, 40.123000000000, 40.123000000000, 40.123000000000, 0.000000000000, 0.000000000000] [6.017000000000, -1.200000000000, 1.200000000000] [66.110600000000, 79.118500000000, 300.343000000000, 44.115600000000, 45.158700000000] \N [-1.2000, 300.3430, 36.1155, 0.0000, 27.0953] [] [17.1499, 35.1627, 40.1230, 100.0200, 40.1230, 100.0200, 80.0718, 37.0727] [-1, 92, 0, 300, -1, 90, 40, 15] \N [85, 12, 40, 0, 88] [94, 57, 19] ["going then don't okay back your I'm so was about", "", "提货价格一家等等"] ["--", "尊重", "he could going hey", "he's", "didn't time it's did up he's hey", "请问", "国际企业", "掉线", "can't she no she", "?"] ["--", "伤心就会逍遥这一说科技大厦调货零点人往高处", "some that's did as would", "v", "-"] ["what", "?", "were", "-", "do will with of what are out", "?", "do", "公司的"] ["how how", "if", "我在政治哦"] ["m", "this", "没变", "h", "?"] [] ["k", "提到人生不了安装", "g", "管理图腾千兆刚刚出去屡禁宫颈糜烂申请人副总生病", "前端解释开通"] ["b", "q", "b"] \N [] ["x", "g", "u", "u", "v", "q", "l", "l"] [2024-01-09, 2014-08-12, 2023-12-18, 2023-12-16, 2024-01-17, 2024-01-09, 2026-01-18, 2025-06-18] [2026-02-18, 2017-08-04, 2026-02-18, 2024-01-19, 9999-12-31, 2025-06-18, 2027-01-09, 2023-12-20] [2024-01-09, 2023-12-10, 2023-12-10] [2025-02-17, 2024-01-09, 2027-01-09, 2014-08-12, 2023-12-19] [2024-01-09 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00] [2026-01-18 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2023-01-15 08:32:59, 2023-12-14 00:00:00] [2023-12-20 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2024-01-19 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2015-03-24 00:00:00, 2012-10-15 00:00:00] [2023-12-12 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59, 2023-12-20 00:00:00] [2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59] [2024-08-03 13:08:30, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2026-02-18 00:00:00] [2025-06-18 00:00:00, 2026-02-18 00:00:00, 2024-06-30 12:01:02.123000] [2024-01-31 00:00:00, 2023-12-16 00:00:00, 2027-01-09 00:00:00] [2023-01-15 08:32:59.123123, 2024-08-03 13:08:30, 2023-12-15 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2009-02-14 00:00:00] [2024-01-17 00:00:00, 2018-12-22 00:00:00, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-14 00:00:00] [2023-12-20 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123123, 2027-01-09 00:00:00] +2 \N 375274506 -225853945 646958674 2024-01-08 2025-06-18 2024-01-09 2024-01-09 - ? or know as now -- 中关村 伟泽财务部南阳路含税天和带你 would he I'll hey had then because say 在乎 0 0 32.119800 100.020000 24.156100 40.123000 2023-12-18 00:00:00 2023-12-12 00:00:00 2024-01-09 00:00:00 2023-12-19 00:00:00 2025-06-18 00:00:00 2023-12-19 00:00:00 2027-01-16 00:00:00 2027-01-16 00:00:00 [0, 1, 0] [0, 1, 1, 1, 0] [0, 0, 1, 1, 1] [1, 1, 0, 1, 1, 0, 0, 0, 0, 0] [-1, 85, 127, -128, 23, 127, -1, 1] [28, 1, 1, 127, 0] [] [-11, 127, -108, -1, -1] [-19059, -24829, -29809, -1, 245, 1, -1, 32767] [8510, 0, 32767, 1, -1, -32768, 0, 0, -1, -1] [-1309, -1, 32767] [] [] [-542558327, -944679110, -1140127664, 1330212540, -1095186866] [] [1, -2123074031, 955726085, -2147483648, -2147483648, 0, 1, 1, -1, -2147483648] [147483648, -2121988440, 1339558288368053247, 9071022920353960524, 147483648, 245, -986225, 147483648] [] [0, 1, 1, 2491823606866231759, -9223372036854775808] [2335211148256714623, -9215146402707467830, -391120, -9223372036854775808, 6194005028188103374] [2, 0, 7, 8, 0, 0, 1387529731, 3] [1, 8, -1972244193, 7, -2, -2, 1, -1, -362072513, 8] [6, -122748902, 6, 2, 1, 3, 0, 3, -2, 1068832748] [1, 5, 1, -1, -172008818] [93.196800000000, 100.020000000000, 1.200000000000, 40.123000000000, 96.154000000000, 0.000000000000, 55.083600000000, 100.020000000000, 92.012100000000, 1.200000000000] \N [21.068500000000, -1.200000000000, 300.343000000000] [] [] [1.2000, 300.3430, 40.1230, -1.2000, 40.1230, 100.0274, 300.3430, 300.3430, 0.0000, 93.0294] [43.1479, 40.1230, -1.2000] [40.1230, -1.2000, 40.1230, -1.2000, 97.1769, 59.0984, 54.1915, 34.1067, 83.1881, 100.0200] [65, 25, 84, 72, 61] [] [50, -1, -1, 0, 1, 300, 300, 52] [78, 40, -1, 40, 60, 1, 100, 0] ["扬子资格中小型停产指出来", "?", "你来找我吧", "查过周保全安全主机节前牌子凡是不可李平残品", "hey", "-", "will", "q"] ["him", "c", "that", "", "铭岳下个去年中天雪松路对咱们购买专区壹仟伍", "on", "联硕招商内外潍坊做好厦门文档", "p"] ["?", "x", "华康", "故事", "on this you're back time with well look been", "", "爱国", "right", "订单", "-"] ["刀片服务器", "just", "不分介绍电视台烟火请你见过面哭泣不谈游戏", "come can't", "表格宏泰吹去广大中的不难明白以下任性金城"] ["投缘", "tell the", "--", "look", "--", "双电源", "群英", "交叉口哦", "can't", "正常"] ["if", "-", "i or who just can be that's back", "创恒电子地矿研究所支持拿下说声珊瑚版", "领你总经理连接线离谱不懂哥哥看看智慧", "will or of could can't then go tell can't", "well", "hey that didn't to be look are here", "see my will get want was have", ""] [] ["也要", "--", "对应目标线缆下单周经理查证比较一开不方便"] \N ["y", "h", "s", "i", "u", "i", "b", "c"] ["r", "q", "y", "k", "l"] ["o", "p", "s"] [] [2024-06-30, 2023-12-10, 2014-08-12] [2023-12-10, 2024-01-09, 2024-01-09] [2023-12-15, 2025-06-18, 2023-12-16, 2024-01-19, 2025-02-18, 2023-12-20, 2023-12-17, 2024-01-31, 2024-01-09, 2024-01-08] [2023-12-19 00:00:00, 2023-12-10 00:00:00, 2023-12-19 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00] [2024-01-08 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2024-01-08 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2023-12-11 00:00:00] [2024-01-19 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2023-12-10 00:00:00] [2024-08-03 13:08:30, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00] [] [2014-08-12 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 23:59:59, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2009-06-27 00:00:00, 2025-06-18 00:00:00] [2023-12-11 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-19 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [2023-01-15 08:32:59.123123, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-17 00:00:00] [2024-08-03 13:08:30, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00] [2023-12-16 00:00:00, 2024-01-09 00:00:00, 2001-05-24 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] +4 998031650 \N -2147483648 715270864 2023-12-13 2025-06-18 2026-02-18 2024-01-31 who 基地人员回首单核们认识儿时 u 陌生人 why good yeah good on had why back who why she for on yeah was because think some here like really so to who then ? 1 0 1.200000 100.020000 300.343000 0.000000 2024-01-17 00:00:00 2023-12-19 00:00:00 2026-02-18 00:00:00 2024-01-17 00:00:00 2023-12-19 00:00:00 2023-12-19 00:00:00 2025-02-17 00:00:00 2025-02-17 00:00:00 [] \N [1, 1, 1] [] [-102, -68, 82, 4, -16] [0, 1, -1, 1, 67] [-28, 127, 7] [35, 9, 127, 1, 1] [32767, -11126, 32767, 1, 27361, 245, 3732, 0, -32768, -24072] [-32768, -29293, 17596, 0, -3854, 19259, 245, 245] [-13694, -1, 20613, 245, 12305, -5776, -1, -1] [1, -26442, 1, 2617, 7167] [0, 147483648, 147483648, 1, -147784245] [-1272117893, 147483648, 745746309, 1, -1, 1, 1376081616, 32679, -1734061611, 1252761569] [305784508, 1074003418, 0, -1774678240, 0, -478897620, -904324480, -462025138] [147483648, -2147483648, 1785325619, 1372878216, 1, -1, 147483648, 2147483647, -1575192374, 2147483647] \N [-1, 9223372036854775807, 9223372036854775807, 147483648, -7039431347334320793, 0, -4219524963407553706, 4334542206478551330] [147483648, 9174468122979065722, -2041516262, -9223372036854775808, -1, 0, 9223372036854775807, 0] [] [-2, 1, 1] [2092036403, 1, -1, 0, 0, -1, 0, -2] [-127632626, -1, 1683714592, -2, -1, 0, 7, 0] [4, -995081284, -1, 727413437, 5, 6, -1, 1, 1, -2] [42.049800000000, 300.343000000000, 300.343000000000, 300.343000000000, 1.200000000000] [100.020000000000, 100.020000000000, 100.020000000000, 20.194900000000, 100.020000000000, 100.020000000000, 41.146300000000, 25.075700000000] [] [1.200000000000, 76.139500000000, 40.123000000000, 0.000000000000, 0.000000000000, 300.343000000000, 51.160800000000, -1.200000000000] [-1.2000, -1.2000, 40.1230] [1.2000, 98.1391, -1.2000] [100.0200, 300.3430, 300.3430] [300.3430, 40.1230, 56.1797, -1.2000, 90.0107, 0.0000, 68.1843, 96.0799] [100, 0, 300, 1, 20, 40, 93, 0, 300, 25] [100, 40, 100] [56, 300, 0, 0, 1, 100, 100, 300, 100, 40] [74, 34, 1, 1, 68] ["had", "z", "i", "-", "-", "just right go him do there really right he", "to got from", "?", "去掉虚妄无限不让农业路位置", "壹万"] ["from", "some", "", "刚刚跑跑卡丁车", "细节"] ["h", "e", "?", "提成点大雪付出过", "还不错"] ["--", "周鹏", "?", "它是", "--"] ["one come don't him who I'll on want", "--", "see didn't well up all", "制作", "愿意龙飞一点要时间各位人往高处重新发"] \N ["--", "her think say you're her me been here in", "but ok you're can't had were for", "could", "--"] ["go", "", "招工"] ["x", "w", "s"] ["d", "c", "i", "j", "g", "i", "e", "v"] ["u", "e", "a", "p", "b"] ["u", "e", "y", "x", "h"] [2023-12-09, 2027-01-16, 2015-07-03] [2023-12-15, 2023-12-11, 2023-01-15, 2026-01-18, 2024-01-19, 2023-01-15, 2026-02-18, 2024-06-30, 2023-12-18, 2025-06-18] [2024-02-18, 2024-06-30, 2025-02-17] [2024-01-09, 2023-12-11, 2024-01-31, 2023-12-09, 2023-12-13, 2023-12-17, 2023-12-19, 2023-12-19, 2027-01-09, 2000-11-22] \N [2023-12-20 00:00:00, 2024-06-30 12:01:02, 2024-01-08 00:00:00] [2023-12-09 00:00:00, 2023-12-12 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2024-06-30 12:01:02, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00] [2023-12-09 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00] [2024-02-18 00:00:00, 2012-10-24 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00, 2008-09-04 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000] [2023-12-10 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [2024-01-08 00:00:00, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00, 2023-12-10 00:00:00] [2024-01-19 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00, 2024-06-30 12:01:02.123000] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-18 00:00:00, 2027-01-16 00:00:00] [] [2024-07-01 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00] [2023-12-19 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00] +5 2147483647 -677311079 -904875204 1 2025-02-18 2023-12-13 2027-01-16 2024-02-18 瑞星新拓我们的琪雪尼奇 相恋 no could look e did and come his her back right of would 方便多少信息领出 ? 1 0 1.200000 \N 0.000000 -1.200000 2024-07-01 00:00:00 2026-01-18 00:00:00 2023-12-16 00:00:00 2023-12-09 00:00:00 2023-12-15 00:00:00 2024-07-01 00:00:00 2024-01-31 00:00:00 2024-06-30 12:01:02.123000 [1, 1, 1] [1, 0, 1, 0, 0, 0, 0, 0, 0, 0] [1, 1, 1, 0, 1, 1, 1, 1, 0, 1] [0, 0, 1, 1, 1] [122, -128, 0, 0, 127] [] [] [0, 0, 1] [245, 32767, -3507] [-11460, 4603, 245, -32768, -32768] [7277, -14182, 17239, 7406, 0, 31691, 0, -31634, -15102, 6419] [245, -14052, -2250, -1, 1030] [32679, -1, -1] [1083267445, -681282890, 0, -1894967634, -1474519650, -1, -518889900, 1] [2064873552, -1454999156, 180624366, 2147483647, 32679, 756572917, -786049768, 2147483647] [1212986934, -1280499872, 32679, 1351628043, 147483648] [-1355372493, 147483648, -1836091008089471008, -1, -1] [9223372036854775807, 1, 1, 9223372036854775807, 1, -4598461, 147483648, 282845784415117631, 9144086299732238603, 0] [1, -3118353, -9223372036854775808, 147483648, 9223372036854775807] [-7918109013005931167, 147483648, -489948814908150392] [-1, -2, 0, 0, 1, -2, 0, 0, 1, -2] [474917001, 0, -1, 1131945655, 861029065, 1, 1181540752, 0] [1, 1, 1261299539, -1, 1] [1, 1, -2006971437, -2, -2100649231, 0, -1, 1, 9, 1] [-1.200000000000, 38.131600000000, 300.343000000000, 0.000000000000, 100.020000000000] [40.123000000000, 1.200000000000, 100.020000000000] [19.186500000000, 14.175800000000, 1.200000000000, 100.020000000000, 1.200000000000] [36.016000000000, 29.075900000000, 40.123000000000, 1.200000000000, 1.200000000000] [] [] [-1.2000, 45.1676, 57.1088, 1.2000, 1.2000, -1.2000, 1.2000, 7.1446] [23.1861, 4.0678, -1.2000, 2.0105, 20.0130] [100, 0, 2, 12, 19, 29, 300, 78] [1, 40, 300] [23, 68, 19] [300, 0, -1, -1, -1, -1, 40, 1] ["安邦", "f", "-"] ["词语恶心死两天嵩县自信能力汝南总价下了稳定性", "方法天翔", "", "ok", "本科自学考试旁边帐期先这样那样子子卡长大那天"] ["--", "so", "", "-", "?", "did I'll now for his with ok", "", "h", "", "张峰最深活动伴我到"] ["and", "one do", "东方"] ["武汉乙方工商局没有到货空间冗余严重怎么样", "I'm come like", "t", "-", "", "some him", "look up I'm you're", "聂柯走在待定", "营销抓住取回日常王总姐姐参数手册通讯好早直接费用", "one I'm at like why one it didn't no"] ["-", "", "院内天星不叫捕鲸船孔艳不曾"] [] [] ["w", "r", "t"] ["u", "h", "o", "c", "b"] ["w", "g", "r", "k", "j"] [] [2024-07-01, 2016-08-24, 2023-12-11, 2024-01-19, 2024-01-31] [2024-08-03, 2023-12-19, 2024-01-31, 2023-12-14, 2024-02-18] [2023-12-14, 2024-07-01, 2023-12-13, 2023-12-16, 2026-01-18] [2024-01-17, 2023-12-17, 2023-12-17, 2025-06-18, 2014-08-12] [2014-08-12 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00] [2024-01-31 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00] [] [] [2025-06-18 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00] [2023-12-11 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2003-09-19 00:00:00, 2024-01-08 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-17 00:00:00] [2025-06-18 00:00:00, 2023-12-17 00:00:00, 2002-02-05 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00] [2023-12-09 00:00:00, 2023-12-12 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-15 00:00:00] [2023-12-18 00:00:00, 2024-01-08 00:00:00, 2015-08-14 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00] [2024-01-31 00:00:00, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00] [2023-12-12 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00] +6 1419115826 1747552254 -1 0 2023-12-16 2006-02-18 2023-12-18 2023-12-09 good \N here or see could he's like his a a because have in you're 伟博下载眼睁睁事先相逢可以到农村弯弯的月亮威达还在吗 必须轻轻刘畅新年好可能人民路等级卫生拥有优先 纵横 \N 0 -1.200000 1.200000 42.065400 0.000000 2025-06-18 00:00:00 2023-12-09 00:00:00 2023-12-14 00:00:00 2014-08-12 00:00:00 2024-07-01 00:00:00 2023-12-09 00:00:00 2026-01-18 00:00:00 2023-12-17 00:00:00 [1, 0, 0] [1, 1, 1] [1, 0, 1, 1, 1] [1, 1, 0, 1, 1] [0, 127, 28, 66, -97, -51, -1, 1, 3, -128] [-1, -119, 0, -1, 29, 73, -1, -1] [127, -18, -109, 7, 127, -58, 0, 0] [-1, 1, -128, 58, 1] \N [245, 1, -2328] [-1, 245, 32767, 23849, 20119, 26170, 12104, 27724] [] [-1515131477, 1167858906, 1050043639, 0, 469342979, 0, 2147483647, 147483648, 32679, -208278580] [1, -479893057, -2147483648, 2147483647, 1851597122, 1279070948, -456760533, 1] [1, -1383497293, -2147483648, -1, -1] [] \N [147483648, 928578234268770707, 147483648, 6668650, 147483648, -9223372036854775808, -1, 245, -1, 147483648] [-175420036, 32679, -5599665688890848700, 245, -19395318572846580] [245, 245, -1, 9223372036854775807, 4630164108919792198] [1, -2, 9, 2, -1, -2, -432763197, 420435951, -294144173, -1] [1, -1765788168, -1] [] [1, -2, -1, 5, 1, 0, 9, 1468039919, 1, 3] [0.000000000000, 80.119100000000, 17.145100000000, 39.003500000000, 9.028000000000, -1.200000000000, 100.020000000000, 92.191900000000, 40.123000000000, 100.097500000000] [89.139000000000, 54.195000000000, -1.200000000000, 65.069300000000, 1.200000000000, -1.200000000000, 1.200000000000, 11.065100000000] [] [100.020000000000, 300.343000000000, 40.123000000000, 80.050400000000, 100.020000000000] [19.0444, 0.0000, 100.0200, -1.2000, 33.0111] [] [100.0952, 40.1230, 300.3430, 0.0000, 0.0000] [0.0000, 0.0000, -1.2000, 1.2000, 42.0015] [-1, 40, 59, 0, -1, 1, 0, 40, 100, 21] [300, 300, 300, 40, -1] [] [0, 70, 97, 0, 24] ["", "?", "get here I'm now here me are out", "and from come", "停留金牌下面时候要求特配机器深圳", "one that come tell right not not my that will", "t", "其中欧卓越才到询问没人出入"] ["", "don't get what not had him your", "", "-", "可以到处理一下哭泣向此", "yes", "选择", "the she or just a been right what as"] ["?", "操心凡是功能", "about", "网上", "?", "华中", "张先生赞美诗走走", "this"] [] [] ["一小收藏夹奇偶向西润泽实现罢了", "--", "it's", "", "-"] ["think", "空间冗余", "", "if", "失误平志伟困扰着周报", "I'm", "got look you something the he's now been be I'll", "don't think in go of ok as see will at", "--", "tell for my he that's we"] ["s", "风格好人", "x", "顾星文宁哦跟你说健民人怜中成官方新世纪", "", "", "不想", "?"] ["u", "f", "v", "w", "c", "o", "d", "i"] ["y", "r", "a", "c", "i"] ["j", "n", "f"] ["t", "i", "u", "o", "r", "a", "z", "w"] [2023-12-12, 2024-01-31, 2023-12-11, 2023-12-13, 2026-01-18, 2023-12-19, 2023-12-20, 2024-06-30, 2027-01-16, 2024-01-08] [2024-01-17, 2025-02-18, 2023-12-19, 2023-12-09, 2024-01-08, 2023-12-09, 2025-06-18, 2023-12-19] [2025-02-18, 2014-08-12, 2025-02-17, 2025-06-18, 2025-06-18, 2023-12-16, 2025-02-17, 2014-08-12, 2023-12-15, 2023-01-15] [] [2023-12-15 00:00:00, 2023-12-15 00:00:00, 2012-06-24 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00] \N [2023-12-20 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [] [2024-01-19 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-08-03 13:08:30] [2024-08-03 13:08:30, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2018-02-01 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30, 2008-04-04 00:00:00] [2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00] [2023-12-11 00:00:00, 2023-12-13 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00] [2024-01-17 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2024-01-08 00:00:00, 2019-02-20 00:00:00, 2024-06-30 12:01:02.123000, 2014-08-12 00:00:00] [2024-01-31 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00] +7 -2147483648 2041660160 -2147483648 1 2024-01-08 2023-12-11 2024-07-01 2026-01-18 医疗 y she for so are if you then she -- oh a but say we have -- was don't \N 1 59.190300 37.141900 4.144700 -1.200000 2023-12-15 00:00:00 2024-01-31 00:00:00 2023-12-10 00:00:00 2023-12-11 00:00:00 2023-12-10 00:00:00 2024-01-08 00:00:00 2023-12-15 00:00:00 2024-01-09 00:00:00 [1, 1, 1] [1, 1, 0, 0, 0] [1, 1, 0] [] [-72, 57, 1, -26, 0] [] [-128, -1, 86, 1, 127, -65, 89, 4] [] [-1, 10374, -32768, 0, 19661] [-22407, -12981, 30404, -19562, -14974, -28514, -16233, 13606, -20814, -15129] [-1, -16488, -13337, 245, 19115, 7177, 0, 245, 245, -2725] [-1, -12813, 2759] [-217202244, 0, 1602973164] \N [-1, -1, -1557949074, 1, 2147483647] [] \N [0, 32679, 147483648, 245, 32679, 1110454858329068800, 245, 1] [4364267, -1, -9223372036854775808, -8221212, 245, 147483648, 9223372036854775807, -765613572391273680] [0, 2800433, -4527029982859207386, 147483648, 147483648, 32679, -1, 1] [-1, 106304555, -1] [7, -22164061, 72271381] [1059906249, -1938750771, 742632731, -2, 0, 7, -4294259, -2] [0, 0, -2, -2, 0, -1245119568, 1, 0, -1, -1] [78.194900000000, 85.104900000000, 1.200000000000, 1.200000000000, 88.079100000000] [62.159800000000, 37.020800000000, 100.004500000000, 24.023400000000, 51.140600000000, 1.200000000000, -1.200000000000, 1.200000000000, 100.020000000000, 100.020000000000] [1.200000000000, -1.200000000000, 40.123000000000, 0.000000000000, 100.020000000000, 100.020000000000, 100.020000000000, 1.200000000000] [4.068700000000, 0.000000000000, 88.062400000000, 40.123000000000, 100.020000000000, 40.123000000000, 82.122500000000, 39.098200000000] [13.0947, 0.0000, 300.3430] [80.1981, 100.0478, -1.2000, 19.1950, -1.2000] [31.1499, 0.0000, 0.0000, 1.2000, 100.0200, 100.0200, 67.0167, 0.0000] [74.0998, 0.0000, 89.0728, 14.0346, 54.1050, 82.0586, 300.3430, 100.0200, -1.2000, 300.3430] [300, 40, 100, 0, 3, 0, 1, 100, 300, -1] [1, 27, 16, 30, 300, 1, 300, 100] [] [70, 45, 41] ["go", "that's", "参数手册"] [] [] [] ["亮度", "去不了请你银联有一天看着赫尔", "do then if up at well a been will", "?", "-"] [] ["中天不方便华栋经销母亲", "oh", "--", "no", "k", "", "I'm", "-", "正规", "圣辉高级自私添加很深你杀时间回来软件园不爱护哦"] ["场景", "--", "钢琴曲使人第二次"] ["a", "n", "k", "r", "x", "d", "b", "a"] ["w", "u", "n", "k", "l", "y", "n", "g"] [] ["e", "b", "p"] [2023-12-16, 2025-02-18, 2023-12-15, 2023-01-15, 9999-12-31, 2026-01-18, 2026-02-18, 2024-01-31] [2024-01-17, 2023-12-10, 2023-12-15, 2027-01-09, 2023-12-11, 2023-12-18, 2027-01-09, 2023-01-15] [] [2026-02-18, 2025-02-18, 2023-12-16, 2024-01-19, 2024-07-01, 2023-12-18, 2024-01-31, 2024-01-08, 2014-08-12, 2024-02-18] [2002-06-06 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00] [] [2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-08-03 13:08:30] [] \N [2027-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-15 00:00:00, 2024-01-17 00:00:00, 2023-12-12 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00] [] [2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123000, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00] [] [2027-01-16 00:00:00, 2023-01-15 08:32:59.123123, 9999-12-31 23:59:59, 2023-12-11 00:00:00, 2024-08-03 13:08:30, 2004-02-19 00:00:00, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00] [2023-12-19 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00] [2024-01-31 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00] +8 -2029217322 32679 -2147483648 1725052390 2024-01-31 2023-12-10 9999-12-31 2023-12-11 ? 不论 - \N - 质保金想过电话给我恢复 could up some I'm I'm now don't 责任谁做张小燕安排好 \N 1 0.000000 0.000000 -1.200000 1.200000 2024-01-09 00:00:00 2023-12-20 00:00:00 2024-02-18 00:00:00 2023-12-15 00:00:00 9999-12-31 23:59:59 2024-01-17 00:00:00 2014-08-12 00:00:00 2024-01-19 00:00:00 [0, 1, 1, 0, 0, 0, 0, 0] [] [] [0, 0, 1, 0, 0] \N \N [] [108, 1, 127, -25, 127, 1, 47, -1, -1, -128] [-12538, 10971, -13948, -31785, 32767, 32384, -22586, 245] [1, -32448, -32216, 1, 0] [-26307, -32768, 23437, 0, 32767, -32768, -1, -14362, -24243, 0] [-6991, 28614, 15737] \N [-592373799, -1095212259, -2109896153, -2147483648, -1, 40045562, 758170033, 1488542082, 565904034, 32679] [1021099247, 0, 1] [2122545364, 2147483647, -657618272] [0, -6432899280565512213, -6730684620887923462, 3745578090713529895, 32679] [] [-2332958981952449393, 147483648, 3656233405564259315, 8195014, 9223372036854775807] [] [-266384322, -1, 0, 1332887750, -277203957, 8, -2, 1, 6, 9] [] [1, 1, 4, 1, 1, -2, 1, -1] [1, -1, -2, 653272908, -2, -1, 4, -1] [] [30.173800000000, 1.200000000000, 92.045800000000, 300.343000000000, 31.163000000000, 40.123000000000, 1.200000000000, 40.123000000000] [85.134100000000, 58.057600000000, 40.123000000000, 300.343000000000, 96.089200000000] [-1.200000000000, 1.015400000000, 300.343000000000] [0.0000, 53.0739, 0.0000, 15.0242, 40.1230, 40.1230, 100.0200, 63.0456] [] [100.0200, 1.2000, 1.2000] [] [40, 0, 40, 40, 7] [8, -1, 300, 0, 100, 6, 1, 300, -1, 53] [1, 86, 27, 100, 0] [1, 300, 42, 92, 1, 14, -1, 300, 300, 300] ["f", "that", "?", "-", "but him at that's to think will"] ["弟子", "唯一的不再", "my"] ["hey", "", "季度太原不太"] ["东西诚信离职", "?", "-", "would to because her", "只是", "", "请到", "?"] ["--", "l", "平顶山一舟下个坐坐党经理因为操心套多海天高杰", "--", "--", "", "know some had are you're can't", "近台南世上杂请你们单价"] ["", "--", "-", "--", ""] ["-", "一声正道", "i", "say", "--", "自己", "ok ok good just oh as in good", "高杰"] ["with", "know", "-", "", "can my for"] \N ["k", "w", "x", "h", "z", "z", "e", "i"] ["m", "d", "w", "m", "t", "n", "v", "p"] [] \N [2024-01-19, 2024-02-18, 2023-12-15, 2023-01-15, 2024-01-19, 9999-12-31, 2026-02-18, 9999-12-31, 2023-12-15, 2024-02-18] [2025-06-18, 2025-06-18, 9999-12-31, 2023-12-14, 2023-12-11, 2024-07-01, 2023-12-18, 2023-12-17] [2009-08-23, 2024-01-19, 2024-02-18] [2023-12-18 00:00:00, 2025-02-18 00:00:00, 2013-06-14 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00] [2023-12-19 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00] [2025-02-17 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2024-01-09 00:00:00] [2023-12-10 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00] [2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00] [2023-12-15 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00] [] [2023-12-11 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2002-04-01 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00] [2024-01-09 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59.123123, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2024-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00] [2023-12-16 00:00:00, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-15 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00] +9 649736936 1 1 147483648 2024-02-18 2023-12-09 2023-12-11 2023-12-11 打印机 技术员水哦表格同力确实孙娟说不出事业一下阿 晨晖张华漯河那行刀片服务器硅谷出不了没有货乱七八糟 利落 the - 1 1 300.343000 -1.200000 -1.200000 0.000000 2024-01-09 00:00:00 2023-12-17 00:00:00 2023-12-15 00:00:00 2025-06-18 00:00:00 2023-12-13 00:00:00 2023-12-16 00:00:00 2023-01-15 08:32:59.123123 2024-02-18 00:00:00 \N [1, 0, 1, 0, 0] [0, 0, 0, 0, 1, 1, 1, 0] [0, 1, 0, 1, 1, 0, 0, 1, 0, 1] [127, 127, -4, 7, -1, 33, -4, 3] [127, -111, -82] [1, 1, 7, 127, -128] [127, 127, 56, -128, -3, 0, 107, -128, 53, 0] [] [30108, 1, 1, 32767, 14164] [] [-32768, 245, 0] [1636576636, 2147483647, 48363910, 1407513173, 0] [-1448567817, -778626295, 906521176] [0, 2147483647, -1661773026, -913664032, 245345519, -331090329, 213563541, -1, 893043675, 1028379451] [147483648, 1, -316980835, -1995019421, 2147483647, 32679, 147483648, 2147483647] [] [5251350023163793405, 32679, 0] [-5936940583397253150, -5071476, 147483648, 592388236877064950, 9223372036854775807] [-1, -7451386356699618910, 6062411458047199598, 32679, 1514399375, 9223372036854775807, 1955812310548243589, 433620027, 4448992339618069000, -8267647909765009513] [8, 1, -2] [0, -1427402736, 1] [1, 1, -2] [785351147, 1, 0, -1844811787, -587921397] [0.000000000000, 92.118400000000, 0.000000000000, -1.200000000000, 300.343000000000, 3.125100000000, 74.083900000000, 27.183700000000, 100.020000000000, 0.000000000000] [72.188900000000, 22.024500000000, 100.020000000000] [40.123000000000, 0.000000000000, 40.123000000000] [-1.200000000000, 40.123000000000, 43.169600000000, 98.063900000000, 1.200000000000, 46.084100000000, 29.088200000000, -1.200000000000, 100.020000000000, 40.123000000000] [1.2000, 100.0200, 1.2000, 80.0977, 40.1230] [71.1675, 300.3430, 1.2000, 100.0200, 100.1117, 0.0000, 46.1719, 31.0381, -1.2000, 40.1230] [] [-1.2000, 0.0000, 300.3430, 79.0939, 100.0200, 300.3430, 100.0200, 40.1230, 65.0519, 1.2000] [1, 0, 0, 300, 100] [88, -1, 40] [63, 33, 100] [0, 54, 8, 0, 40, 300, 40, 40] ["购买专区", "文艺兄弟视听", "you", "-", "车费有缘白明伟开始利万年度摆脱明天扬子美捷", "置业参考无所谓干干净净台式机", "not didn't", "全系列留个最深刻杨宗小徐虽然看电影根本驱动器着呢", "--", "--"] ["成熟", "e", "公司的主奴婢收集嘉运达最大化政府", "", "?"] ["?", "?", "", "验货单那段顶多期限资质赶紧金总麻烦", "because that's some will look that", "right", "say", "--", "折叠近台南大忙人我打印出来个给你吧弱阳性地图", "取悦不想刀片服务器厉害这也是热备软件高亮屏出纳实现航务"] ["all", "who", "yeah", "?", "-", "问过重要", "-", "-", "广告词", "?"] ["when", "that for could but was no he on there", "l"] ["look", "well good here like just", "something", "this he I'm", "--"] ["高密度供货商结业捷创性价比汉语李峰多少钱合力吸纳中", "d", "this from could can't did", "授权存储唯一", "支原体附件样机位居王峰", "r", "一切", "?", "about", "n"] ["-", "还在吗", "?", "-", "just", "出去", "屏幕", "?", "--", "-"] ["a", "e", "t", "t", "m", "m", "g", "o"] \N ["m", "d", "c"] ["c", "w", "n"] \N [2023-12-20, 2027-01-16, 2026-02-18] [2024-08-03, 2023-12-12, 2023-12-20, 2023-12-14, 2026-01-18, 2024-07-01, 2024-02-18, 2005-05-09, 2024-01-19, 2027-01-09] [2024-02-18, 2024-01-08, 2024-02-18] [2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00] [2023-12-12 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00] [] [2025-02-18 00:00:00, 2024-01-19 00:00:00, 9999-12-31 23:59:59] [2023-12-13 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00] [2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59.123000, 2026-01-18 00:00:00] [2023-12-09 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00] [] [9999-12-31 23:59:59, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00] [2024-01-08 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00] [2024-01-19 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00] [2009-10-16 00:00:00, 2023-12-16 00:00:00, 2024-01-31 00:00:00, 2024-01-08 00:00:00, 2024-08-03 13:08:30] +10 19564813 279915089 256035 -1 2025-02-17 2027-01-09 2023-12-12 2024-01-09 - this oh who out it you're that's be have all of as then all the like you're 看你 we up -- 1 0 91.002200 35.172000 0.000000 0.000000 2024-01-08 00:00:00 2023-12-15 00:00:00 2024-01-17 00:00:00 2027-01-16 00:00:00 2023-12-15 00:00:00 2026-02-18 00:00:00 2024-01-31 00:00:00 2019-10-18 00:00:00 [] [1, 1, 1] [1, 1, 0] [1, 0, 1, 0, 1, 0, 1, 1] [0, 0, -96] [-1, -128, 103] [] [0, 2, -128, 44, 127, -1, 7, -73] [-23361, 245, -24695, -1, -1, -32768, -1, 245] [15760, -3050, -13475, -18806, -32768, 0, -8308, 245] [-32768, -32768, 0, -2667, -31458, 14825, 214, 0, 245, 0] [-1, 0, 245, -21773, 32767, -8759, -26944, 32767] \N [147483648, 2147483647, 0, 32679, 1543699419, -1950863925, 1308238653, -738417695] [1445319692, 32679, -1899391767] [-2147483648, 1, 1064156116, -1230319713, -1, -517601122, -275392287, 677742173] \N [9223372036854775807, -1, -889927542] [9223372036854775807, 147483648, 1208251262023788911] [9223372036854775807, 0, 245, -3060355288379481717, 1, 2066044848, -1, 1305451, 9223372036854775807, 784519994] [1, -599611219, 1, -2, 1] [-1, 3, 1719569196, -1, 0, 1344226246, -93401326, -2] [] [1, 0, 2058049572, -1, -2, 541405725, 1513937869, -1, -1181763232, 2] [-1.200000000000, 0.000000000000, 1.200000000000, 0.000000000000, 40.123000000000, 0.083900000000, 100.020000000000, 38.138900000000] [0.096900000000, 0.000000000000, 1.200000000000, 300.343000000000, 1.200000000000, 100.020000000000, 100.020000000000, 0.000000000000, 41.062700000000, 100.020000000000] [91.094900000000, 300.343000000000, 1.200000000000] [36.001300000000, 0.000000000000, 44.107100000000, 40.123000000000, 300.343000000000, 40.123000000000, 96.133200000000, 0.000000000000, 40.123000000000, 1.200000000000] [] [300.3430, 0.0000, 92.0094] [74.0449, 300.3430, 0.0000, 1.2000, 0.0000, 300.3430, 30.1678, 22.0650] [] [84, 100, 0, 1, 1] [] [100, 100, 33, 97, 100, 51, 0, 100] [] ["was", "-", "向你哥哥忘记创新融入刘亚子卡有空热卖", "?", "梁英", "爸妈鸿锐写吧控制卡有限公司硬盘", "单号", "脱机", "oh", "?"] ["-", "r", "说不新年瑞贺越来越来自己总结", "心连心申请粘贴知道了领你命名一想收藏夹什么", "?", "-", "性格天下无贼能力一共随便看来", "另存为"] [] [] ["--", "一是不分背景老板具体你来办理把握三级我们沈总电源", "u"] ["大姐东风系列不论无锡青少年你们开始热线去年几个", "占到", "-", "it when when my was did some there it's get", "y", "", "你来找我吧", "w", "累了", "--"] ["", "如果", "有机会", "but get", "", "then", "", "--"] [] ["z", "k", "w", "l", "t", "u", "f", "d"] ["z", "s", "c", "k", "l", "e", "o", "p"] ["j", "t", "m"] ["r", "x", "a", "s", "r"] [2024-01-19, 2019-08-10, 2023-12-19, 2023-12-11, 2025-02-18, 2023-12-09, 2023-12-09, 2024-07-01] [2024-01-31, 2024-02-18, 2024-01-31, 2024-08-03, 2024-01-09, 2026-01-18, 2023-01-15, 2023-12-12, 2023-12-15, 2023-12-15] [2024-01-08, 2023-12-15, 2011-02-17, 2024-01-17, 2027-01-09] [] [] [2024-06-30 12:01:02, 2024-06-30 12:01:02, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00] [] [2023-12-17 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00] [2024-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-17 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-19 00:00:00, 2023-12-20 00:00:00] [2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00] [2023-12-16 00:00:00, 2023-12-20 00:00:00, 9999-12-31 23:59:59] [2009-08-14 00:00:00, 2023-12-11 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [9999-12-31 00:00:00, 2023-12-17 00:00:00, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59] [2023-12-15 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00] [2026-01-18 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2026-02-18 00:00:00] +12 0 1973584992 1165291145 2147483647 2025-02-17 2023-01-15 2023-12-18 2023-01-15 多多 整理方连 for was could because and have there there look 农行 tell ? b 除外 \N 0 0.000000 40.123000 75.168200 -1.200000 2024-01-09 00:00:00 2023-12-11 00:00:00 2026-01-18 00:00:00 2023-12-12 00:00:00 2011-05-07 00:00:00 2024-02-18 00:00:00 2023-12-15 00:00:00 2023-12-18 00:00:00 [1, 0, 0] [1, 1, 1, 0, 1, 1, 1, 0, 0, 1] [1, 1, 0] [1, 1, 1, 1, 1] [43, -112, 1] [-128, 3, -128, 0, 126] [0, 85, 44, 111, 1] [] [3395, -4101, -4728, -6116, 245, 32767, 245, 13539, -14535, -23070] [-1, 2734, -1, 32767, 13818, -32768, 20370, 1, 1, -27140] [245, 6081, 29402, 16200, 25372, 14088, -32768, -183] [-25692, 14960, 30923, -2264, 2664, 1, 30074, 8179] [32679, -444273079, 1492832640, 1630321542, -2147483648] [] [-2147483648, 32679, 32679, 32679, -2147483648, -1787273246, 0, 0, 147483648, 141320128] [-455051090, 32679, 1049794294] [7407011, 9223372036854775807, 1] [1, 32679, 0] [1, 0, -6704189726394635300] [-3604872527409440099, 1, 32679, -1577180, -9223372036854775808, 1, 1137835778, 1322564030, 147483648, 245] [0, 1429573308, 1, 1, 0] \N [-2, 0, 1, -2, 0, -2, 1, -1043580130] [-1, -2, 0] \N [0.000000000000, -1.200000000000, 0.000000000000, 35.064800000000, 7.162400000000] [55.007400000000, 98.106500000000, 0.000000000000, 100.020000000000, 0.000000000000] [61.097000000000, 15.047500000000, 300.343000000000] [42.1521, 97.0563, -1.2000, 37.0435, 40.1230, 1.2000, 1.2000, 19.1734] [100.0200, -1.2000, 88.0604, 0.0000, 0.0000, 91.0097, 63.1463, 46.0418] [86.1768, 0.0000, 1.2000, -1.2000, 1.2000, 100.0200, 75.1838, 68.1423, 0.0000, 300.3430] [0.0000, 40.1230, 29.0968, 300.3430, 100.0200] [97, 72, 40] [100, 40, 100] [-1, 0, 0, 100, 95, 91, 17, 100] [] ["your time at on that's I'm something i to", "--", "k", "下游发货百川名次老刘官方福利五号", "", "?", "ok", "now on they", "be", "节日协奏曲季度经过哭泣不敢畅谈手续删除供应商"] ["-", "with like about well tell just", "were about hey they her", "很细那个还好吧新增信任意见人家", "could who out", "I'm at yes your could to my so back will", "", "time one something and yeah of as"] ["--", "美好考前难缠伤心心里参数身份证在家", "with", "h", "", "o", "n", "j", "are some say can't as like really come who", "不应该新亚非刀片服务器"] ["未税同感分销商名单休息不再礼拜一孙瑞霞淡季", "b", "背景", "-", "me you go", "刚出门", "多多", "yes him or up you think can't it's come", "there yeah", "--"] ["--", "be it go", "?", "?", "那天部分安阳恢复嘉运达", "-", "-", "b"] ["think", "a", "之道", "提货方法庙里不说面前浮动", "同志们", "", "", "some been who good say think out will mean hey"] ["一年中庙里多大好了不要玩玩好好小曹神秘人情", "w", "--", "he's his", "-", "舞阳老板想象大兵美好心里的天天", "--", "实现", "up", "go"] ["his", "?", "a her we just had then want", "?", "--"] ["x", "y", "z", "t", "n"] ["l", "s", "x", "e", "z"] ["t", "r", "z"] ["d", "f", "g", "s", "z", "y", "f", "q"] [] [] [2023-12-20, 2024-01-31, 2027-01-09, 2023-12-12, 2025-06-18, 2023-01-15, 2023-12-14, 2024-01-08, 2024-01-08, 2023-12-10] [] [9999-12-31 23:59:59, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00] [2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00] [2024-01-19 00:00:00, 2024-06-30 12:01:02, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2023-12-10 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02] [2026-02-18 00:00:00, 2027-01-16 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00, 2003-06-10 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00] [2027-01-16 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00] [9999-12-31 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2025-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00] [] [] [2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123] [2024-01-31 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-08-03 13:08:30] [] [] +13 -354561878 -1940553162 -1963454595 -2147483648 2023-12-13 2027-01-16 2026-01-18 2025-02-17 -- I'll 全款渠道不叫联硕有什么一部分去过请客我刚来呀前辈 for for to of can - want and that for yes when that's you not 除非 0 1 45.172400 0.000000 9.162600 0.000000 \N 2025-02-17 00:00:00 2023-12-12 00:00:00 2023-12-18 00:00:00 2023-12-12 00:00:00 2027-01-09 00:00:00 2025-02-17 00:00:00 2026-01-18 00:00:00 [1, 1, 1, 0, 0] [0, 0, 1, 0, 1, 0, 0, 0] [1, 0, 0] [1, 1, 1, 1, 0] [1, -50, 79, 7, 12] \N [-128, 59, -128] [-1, 8, 127, 127, 0] [-5449, -25212, 0] [-26658, -32768, 32767, 0, 20946, -336, 245, -4447] [1, 22971, 18201] [245, -1, 32767, 32507, 1, -3417, 1589, 1] [147483648, 0, 1] [-1745699812, 2147483647, 1865964462, -81442740, 32679, -1366532107, 147483648, 32679, 1425268893, 1] [580769975, 2147483647, -1015077292, 1388260827, 1] [] [] [] [9223372036854775807, 0, 147483648, 7914888203952013049, 32679, 147483648, 32679, 9223372036854775807] [-923269595, -1, 5787572, -1405686134, 245, 9223372036854775807, 1, 5455502, 9223372036854775807, 0] [1, 0, 2, -1, 1] [0, 1, 0, -2, 960772453] [87980294, 1, 1127929323, -1, -1, -2, -1, -2] [-1026036717, -1490480776, 1, 9, 1, 1, 0, 0, 0, 1] [40.123000000000, 1.200000000000, 37.102600000000, 300.343000000000, 100.020000000000, -1.200000000000, 300.343000000000, 69.146100000000] [300.343000000000, -1.200000000000, 91.054400000000] [] [1.200000000000, 1.200000000000, 73.130400000000] \N [40.1230, 100.0200, 100.0200, 1.2000, 300.3430, 40.1230, 68.0840, 18.1826, 0.0000, 56.0458] [-1.2000, 100.0200, 1.2000, 0.0000, 100.0200, 4.1841, -1.2000, 1.2000, 21.0508, 0.0000] [89.1647, 80.1281, 91.1447, 40.1230, 300.3430, 1.2000, 300.3430, 17.1893] [-1, 2, 40, -1, 0, 57, 40, 300, 100, 0] \N [] [] ["it", "something with say oh they i", "we", "", "", "之夜亲戚每天下次吧随便你是谁鑫创", "", "-", "been if", "--"] ["", "it were didn't all", "g"] ["?", "-", "助理五星提高数据", "瑞昌", "see so all ok you his she want are", "now", "相对", "?"] ["思科", "委托收款", "the we", "with", "好的不在哦不对外大量现货重要写下单条惊喜总代理商初八"] ["but", "", "-", "k", "then", "g", "地球人满天飞谢晓娟五洲", "--", "--", "沉住气"] \N ["", "拿住模块阿奎那焦作舒展", "留个"] ["did", "t", "of me"] ["k", "s", "n", "y", "y"] ["q", "f", "t", "q", "f"] ["q", "j", "n", "z", "x", "k", "t", "g"] ["q", "s", "y", "y", "t", "k", "t", "i"] [9999-12-31, 2023-12-18, 2027-01-09, 2025-02-18, 2024-07-01, 2025-02-18, 2023-12-09, 2024-01-09] [2023-12-15, 2023-12-15, 2023-12-10] [9999-12-31, 2023-12-14, 2027-01-09, 2023-12-18, 2023-12-18] [2024-02-18, 2023-12-12, 2023-12-15] [2024-06-30 12:01:02, 2025-06-18 00:00:00, 2024-06-30 12:01:02] [9999-12-31 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [] [9999-12-31 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00] [] [] [2023-12-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-13 00:00:00] [2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123000, 9999-12-31 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000] [2024-01-31 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00] \N [9999-12-31 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00] [] +14 1454216481 -2147483648 973628695 1 2023-12-20 2023-12-09 2023-12-14 2023-12-17 were look I'm then is 母亲 yeah no could tell I'll going I'll his - ? one to his good 华阳 come you can him 1 0 44.153600 \N 100.020000 28.182700 2024-01-09 00:00:00 2026-02-18 00:00:00 2026-02-18 00:00:00 2025-06-18 00:00:00 \N 2025-02-17 00:00:00 2024-01-08 00:00:00 2014-08-12 00:00:00 [0, 0, 1] [] [0, 1, 1] [] [127, -60, -8, -128, 0] [1, -31, -24, 1, -1] [7, -1, -104, 1, 1, 1, 1, 81, 1, 96] [0, -128, 1] [-32768, -16489, 16397, 32767, 32138, -1, 1, 27322, 26790, 0] [-23273, 245, -13824] [-24600, 32767, 1, 245, -31913, -12913, 245, 19945] [] [-1, -1314058755, 2147483647, 327812055, -2147483648, 0, 161308441, 1959793426, -1, -1685411006] \N [147483648, -1, -1] [] [-1, 9223372036854775807, -1] [-9223372036854775808, -1, -9223372036854775808, 1844341105061323910, -839469448105216992] [147483648, -2036011285, 147483648, 2086836797, 9223372036854775807, -7349522811880187704, -5959643, 245] [-980791, 245, -364107505, 9223372036854775807, -9223372036854775808] [7, -406028574, -2] [1, 1, 1, -751809413, 1, -1, 1, -2, -2, -2] [-1, 6169020, 1812842176, 1, 2, -2, -1, 7, 0, 1] [1046012910, -1, 0, -1, 1, 1, -1, 0] [] [-1.200000000000, -1.200000000000, 300.343000000000] [-1.200000000000, 1.200000000000, 85.122000000000, 300.343000000000, 1.200000000000] [50.181100000000, 1.200000000000, -1.200000000000, 19.041600000000, -1.200000000000] [47.1982, 300.3430, 300.3430, -1.2000, 89.0910] [] [] [1.2000, 38.1377, 32.1242, 1.2000, 27.1518, -1.2000, 1.2000, 1.2000, 12.0040, 11.1855] [] [82, 0, 1] [0, 38, 40, 1, 23, 0, 1, 100, 0, 100] [40, -1, 100, 0, 26, 23, 100, 71] ["then", "科技市场美女地方周保全王瑞相识不到位不定", "你也来了人工湖红颜情歌实话试过亿欧元启动请你吃饭", "?", "all", "", "?", "", "对话框", "can"] ["ok my who really the do well", "like", "do", "my", "王佩丽"] ["主做", "-", "上次", "?", "his something we be I'll back"] ["something if", "r", "安徽", "-", "结识", "女孩承认刘海海域哦听不懂干干净净", "-", "-"] ["哈皮有关我就配置", "自导主做美丽别人不能说之道你不认识航道回公司", "at", "-", "with"] ["hey", "群殴", "?", "i", "q", "?", "?", "m", "联硕数码港焕然了户优势", ""] [] ["?", "her I'm right you're see that's good", ""] ["q", "d", "v"] ["o", "p", "m"] ["g", "s", "u", "q", "u", "r", "j", "q", "f", "y"] ["x", "j", "m", "r", "t", "q", "j", "n", "u", "e"] [2024-07-01, 2024-02-18, 2024-08-03, 2025-02-17, 2023-12-16] [2024-01-19, 2023-12-20, 2026-02-18, 9999-12-31, 9999-12-31, 2026-01-18, 2023-12-17, 2023-12-18, 2023-12-17, 2024-02-18] [2025-02-18, 9999-12-31, 2024-01-19, 2025-02-17, 2025-02-18] [2026-01-18, 2023-12-11, 2024-06-30, 9999-12-31, 2025-02-17, 9999-12-31, 2024-02-18, 2023-12-10, 2027-01-16, 2025-06-18] [2023-12-10 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2012-11-20 00:00:00, 2023-12-16 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59, 2023-12-13 00:00:00, 2024-06-30 12:01:02, 9999-12-31 00:00:00] [2023-12-09 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2024-01-31 00:00:00, 2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-01-19 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00] [9999-12-31 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-17 00:00:00] [2023-12-10 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-10 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00] [2023-12-09 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2025-06-18 00:00:00] [] [] [2026-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-09 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-14 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00] +15 36759907 -1851781108 2147483647 -1 2024-08-03 2014-08-12 2023-12-14 2018-10-23 we me -- don't that one no when really now can't he's 进账 and - - 1 0 61.163700 \N 6.036500 40.123000 2024-01-09 00:00:00 2024-01-09 00:00:00 2023-12-18 00:00:00 2025-02-17 00:00:00 2024-01-08 00:00:00 2024-06-30 12:01:02.123000 2023-12-14 00:00:00 2023-12-17 00:00:00 [0, 1, 0, 0, 0, 0, 1, 0, 0, 1] [0, 1, 1, 0, 0, 1, 0, 0] [0, 1, 1, 0, 1, 0, 1, 0, 1, 0] [1, 0, 1, 0, 0, 0, 0, 0, 1, 0] [1, 1, -128, 69, 8] [] [-49, 0, 5, -1, -1] [-1, 0, -97] [-596, 1, -3189, -11624, -17467, 26513, 7109, -32768, 22734, 0] \N [10286, 8352, 4354, -15220, 24293, 11510, 1, -9] [27705, -1, 1, 9101, 18730, 1, 0, -1713] [1, 1621363168, 1675610118] [-198334257, -1, 1716957267, -2147483648, -819433163] [1997577371, 32679, -799240227] [-1, 2147483647, 762826472, 0, -1, 310955180, -1103322637, 0] [-9223372036854775808, 69944360, -1, -4950838090985850182, 32679] [147483648, 1, 0, -9223372036854775808, -1466144758, 2071072107637311055, -1, 1850755072921226692] [] [-1794610078249537710, -9223372036854775808, 9223372036854775807, 245, 1] [-792395753, -1, 1] \N [-1, 1, -2137288364] [-1, -1, -2, 0, -1, 0, 5, -2] [40.123000000000, 1.200000000000, 0.000000000000, 100.020000000000, 40.123000000000, 57.197700000000, 300.343000000000, 89.188900000000] [300.343000000000, 1.200000000000, 28.013000000000, 54.021300000000, 34.157000000000] [-1.200000000000, 90.108100000000, 40.123000000000, 0.000000000000, 0.000000000000] [82.147600000000, 66.147000000000, 64.090400000000, 100.020000000000, 40.123000000000, 83.077900000000, 100.020000000000, 0.000000000000] [-1.2000, 300.3430, 1.2000] [0.0000, 100.0200, 300.3430, 8.1584, 1.2000, 1.2000, 100.0200, 64.1048, -1.2000, 0.0000] [47.1071, 74.0051, 300.3430] [0.0000, 1.2000, 0.0000, 15.0660, 0.0000, 300.3430, 0.0000, 1.2000] [] [34, 300, 100, 28, 0, 0, 40, 0, 2, 0] [] [72, 40, 100, -1, 0, 40, 1, 76] ["?", "-", "词典", "-", "or a what go will my", "come look here", "--", "人员", "", "in"] ["流程", "t", "待定好不好看不到无法大领导"] ["mean one is we to out but that's like there", "b", "新品普通背景对自己", "", "come a but who can't to about yeah mean", "", "i", "-"] ["网通", "-", "-"] ["would can't see if that's yeah", "z", "--", "this got did just in going", "--"] ["mean with a on oh one I'm", "?", "could yeah had don't", "思科", "-", "-", "-", "-"] ["tell", "--", "?", "if did be had would have that that as", "?", "--", "to", "was"] [] ["t", "v", "n", "x", "h"] ["l", "f", "y", "g", "e", "k", "l", "m"] [] ["q", "s", "h", "t", "q"] [2024-08-03, 2025-06-18, 2024-08-03] [2024-06-30, 2024-01-09, 2024-01-19] [2024-02-18, 2026-01-18, 2027-01-16, 2024-01-17, 2024-02-18] [] [2023-12-09 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00] [2025-02-17 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02, 2023-12-18 00:00:00, 2025-02-17 00:00:00] [2023-12-13 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59, 2024-06-30 12:01:02] [] [2027-01-09 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00] [2025-06-18 00:00:00, 2024-01-08 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-20 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00] [2023-12-09 00:00:00, 2026-02-18 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00] [] [2023-12-13 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2024-08-03 13:08:30] [] [] +16 32679 0 -1 -2147483648 2024-01-08 2024-07-01 2024-07-01 2024-02-18 \N 人民路你杀时间含苞单核惠达同方病毒飞跃 b 新乡 手续没人剪掉万元千兆网卡广角砖石姓名姓名慈利达 惊风 -- \N 1 100.020000 40.123000 300.343000 0.000000 2026-02-18 00:00:00 2023-12-09 00:00:00 2024-08-03 13:08:30 2023-12-12 00:00:00 2024-06-30 12:01:02.123000 2023-12-12 00:00:00 2014-08-12 00:00:00 2024-02-18 00:00:00 [0, 1, 1, 0, 0] [] [1, 1, 0, 0, 0, 1, 1, 1] [1, 0, 0, 1, 0, 0, 0, 0, 0, 1] [-128, 0, -122, -107, 0, 106, 6, 79, 6, 1] \N [0, -38, 127, 4, -128, 0, -128, 104, -62, -1] [-49, 127, 0, 127, 0] [-1, -16528, 501, 8515, -594, 32767, -24980, 32767, -32768, 22448] [-20737, -1, 0, 1370, 29503, 0, 30306, -8809, -1, 1] [-15899, 26794, -27103, -10935, -14214, 1, 245, 0, 0, 1] [-1, 1, -27684, 1, -30637, 4080, -1, 1] [] [2147483647, -1, 32679, 1640913275, 1056968429, 845450275, 2147483647, -1, 385314493, 0] [2147483647, 320352231, 32679, -1797539733, -1208663327, -1269518337, -2147483648, -2147483648, 2147483647, 1258948469] [] [] [245, 7445360720794204147, -1, 32679, 245, -9223372036854775808, 1, 4610178137770285344, -9223372036854775808, -8719686192408322231] [9223372036854775807, 0, -3662990] [3384772463167236880, 8264073787479225252, -4127157214007601080] [] \N [0, 0, -1, -1, -1] [] [] [0.033800000000, 40.123000000000, 100.020000000000] [300.343000000000, 100.020000000000, 12.101900000000] [1.200000000000, -1.200000000000, 1.200000000000] [100.0200, 300.3430, 23.1188, 56.0356, 300.3430, 99.0142, -1.2000, 1.2000, 1.2000, 1.2000] [] [0.0000, 1.2000, 12.0899, 98.1135, 78.1060, 29.1268, 10.1623, 100.0200, 4.1872, 42.1217] [] [100, 40, 40, -1, 29, 28, -1, 47, 40, -1] [300, 31, 4, -1, 100, 1, -1, 1, -1, 60] [14, -1, 73, 40, 1, 48, 300, 40] [100, 40, 81, 19, 76] ["", "--", "know she if look when at her think", "of good back why going at he's get", "some no we come you my was yeah say is", "?", "i", "do that"] ["?", "-", "-", "he's or didn't now really or", "--"] ["q", "--", "经理三石陇海自以为是着呢行货服务器提到之日起"] ["-", "见过面这么第二开封存储北京朝阳区很大派人下次吧缺货", "许昌想过"] ["地方那个现代辅佐相关单个中心站历来申请人货运宽屏", "--", "", "烟厂然后", "", "the and about he time good", "心事王岩春拿不到巴巴有点开出来一下阿", "-", "不懂备案好些可选", ""] ["相隔对自己我给你打吧备份文件发邮箱礼拜量大优惠王青借款肆仟", "防火墙", "-", "不对", "him", "-", "tell as your go going i okay can't", "mean you I'm yeah why know back in what"] ["?", "方便一年力天中原周报千金听不懂备份文件怎么着放心", "争取电脑上空间", "get", "is do good you and will there"] ["月亮我打印出来个给你吧扬子那几", "不小快点之间新山磁盘陪伴所以", "?"] ["v", "m", "d", "j", "p", "j", "p", "p"] ["g", "z", "i", "i", "d"] ["x", "d", "o", "x", "s", "x", "j", "t", "k", "b"] ["e", "f", "w", "f", "v", "j", "f", "c"] [2024-02-18, 2024-02-18, 2023-12-19] [2024-01-17, 9999-12-31, 2027-01-09, 2023-12-20, 2024-01-09, 2024-01-31, 2023-12-15, 2024-02-18, 2024-02-18, 2023-12-16] [] [2008-10-19, 2024-01-19, 2023-12-16, 2023-12-17, 2024-02-18, 2027-01-16, 2024-07-01, 2027-01-16] [2024-07-01 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00] [2023-12-11 00:00:00, 2023-12-12 00:00:00, 2023-12-15 00:00:00] [] [2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00, 2023-01-15 08:32:59, 2024-01-31 00:00:00] [2026-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00] [2024-01-17 00:00:00, 2023-12-16 00:00:00, 2027-01-16 00:00:00] [2023-12-18 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123000] [2023-12-16 00:00:00, 2023-12-13 00:00:00, 2023-12-16 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00] [2027-01-09 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2008-09-04 00:00:00, 2023-12-20 00:00:00, 2023-12-14 00:00:00, 2023-12-20 00:00:00] [2026-01-18 00:00:00, 2024-01-31 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00] [2023-12-19 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2012-02-11 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00] [2023-12-13 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00] +17 2060818624 1116822125 -130359852 32679 2025-02-18 2023-01-15 2027-01-09 2025-06-18 come 出团一般李经理 导轨 me didn't 对话 -- 0 1 31.028100 100.020000 2.197300 72.032200 2026-02-18 00:00:00 2023-12-16 00:00:00 2027-01-09 00:00:00 2025-06-18 00:00:00 \N 2025-06-18 00:00:00 2023-12-13 00:00:00 2024-02-18 00:00:00 [0, 1, 0] [0, 0, 1, 0, 0] [] [0, 1, 0, 0, 0, 1, 0, 1] [127, 1, 1, 1, -38, -4, -128, 0] [127, -1, 0, -1, -79] [-1, 88, -63, -127, -41, 127, -3, -128, 1, 9] [127, -102, 127, 1, -1, 1, 1, -128, -128, 0] [1, 245, 23462, 17449, 245, 26161, 20868, -1] \N [29630, 9296, 9488, 2109, -26927, 4890, 32657, 29272] [] [2147483647, 846987591, -1, -537175561, 0, 2147483647, -104525111, -687102059, 2147483647, -2147483648] [-2071525014, -1, -1902620134, 1531596626, 1] [147483648, -1735266891, 1, -2147483648, -850986090] [2147483647, 153369942, 0, -1, 667183410] [147483648, 9223372036854775807, 32679, -3943541572867799155, 147483648, 1, -5382566759532469243, -1270786062] \N [] [-3268184, 9223372036854775807, 9223372036854775807, 2273638, -7221128643550878935] [-1, -1898305728, -1594339689] [8, -2, 1, 2, -2] [1, 0, 2, 0, 1] [1, -1205438725, 0, -412682820, -2] [90.175800000000, 0.000000000000, 40.123000000000, 1.200000000000, 40.123000000000] [300.343000000000, 1.200000000000, 40.123000000000] [40.123000000000, 100.020000000000, 0.031300000000] [0.000000000000, 0.000000000000, -1.200000000000, 95.052800000000, 94.069900000000] [0.0000, 40.1230, 300.3430, 0.0000, 40.1230, 40.1230, 37.0300, 0.0000] [23.0336, 40.1230, 37.0734, 3.1473, 300.3430] [] [21.0648, 59.1643, 79.1380, -1.2000, 100.0200, 300.3430, 100.0200, 40.1230] [] [86, 2, 38, 9, -1] [62, 40, 300, -1, 1, 69, 87, 96] [100, 40, 88, 100, 7] ["中关村没结束催催查查", "all were from her know my", "m", "杨学限价正信性能可怕", "n", "with", "no", "i", "her be but so time", "up"] ["c", "打开文宁在吗忘记充满", "in see tell some you're had think", "", "with", "公道话授易繁体字独立好的法律王枫", "--", "同力卡巴斯基上机柜天地玫瑰长光瑞昌"] ["总代理", "can hey up him on one like to think", ""] ["do", "h", "城市", "不但是神仙擅作主张数据家庭", "目前"] ["有同学", "ok", "-", "here her", "j", "about why do why had on out can't", "-", "很想外教就在李平你来找我吧世安", "江河两千反应好好哦啊订票贸易通大世界瑞科", "规定你用手机吧胡华威绿城儿童都发生除外"] ["", "一般停产收尾张先生已给", "那就好"] ["--", "think", "?", "-", "about well then did", "", "焕然现在耽误个月过来着呢", "from tell are at up can't well come", "ok", "his if"] ["在他家自明不大一会吧大雪忽视星星", "-", "on they", "周期屏蔽运费一次适合", "", "--", "?", "want want could here are had"] ["o", "j", "z", "x", "y", "m", "s", "h", "z", "k"] ["l", "e", "q"] ["l", "u", "c"] ["b", "r", "u", "d", "r", "x", "m", "i"] [2014-08-12, 2023-12-16, 2024-06-30, 2024-07-01, 2024-01-08] [2024-01-19, 2023-12-12, 2023-12-17] [] [2024-08-03, 9999-12-31, 2023-12-15, 2023-12-16, 2014-08-12] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2003-08-14 00:00:00] [2025-02-17 00:00:00, 2024-01-17 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-13 00:00:00] [2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00] [2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 2023-12-13 00:00:00] [2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2011-08-22 00:00:00, 2025-02-17 00:00:00] [2023-12-14 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-13 00:00:00, 2011-08-07 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00] [2024-01-08 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2025-02-18 00:00:00, 2024-07-01 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00] [2026-01-18 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00] [2026-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2024-07-01 00:00:00] [2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00] [2024-01-09 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00] +18 2147483647 1076719273 -431502320 569022033 2023-12-18 \N 2023-12-11 2027-01-16 下个 - u 徐敬东里面哪一步 ? m i 0 1 40.123000 53.131600 9.142100 100.020000 2023-12-10 00:00:00 2023-12-17 00:00:00 2024-08-03 13:08:30 2023-12-11 00:00:00 9999-12-31 00:00:00 \N 2024-02-18 00:00:00 2023-12-13 00:00:00 [1, 0, 0] [0, 1, 1, 1, 1, 0, 0, 1, 1, 1] [0, 0, 1, 0, 0] [0, 1, 1, 1, 1, 0, 0, 0, 1, 0] [4, 1, -1] [1, 127, 0, 3, 127] [0, -128, -51, -1, -104] [0, 1, 0, 127, -128, -126, 30, 0] \N [413, 9495, -5561, 32767, 1, -1, -8753, -17297, -1, 19321] [-1, -24923, 15837, -17805, 30294, 1, -24789, -32768] [-1, 32767, 245, 1, -32768, 1, -1, 14080] [2147483647, 2147483647, 32679, -213082851, -1768145413] [1240549094, 1972522237, -1, 1, -188724894, -2147483648, -1159773192, -1, 1169161586, -38293117] [-2147483648, -983761505, 2147483647] [283120170, -1651229063, -2147483648, 1316219244, 0, 1653788234, -1140943737, -1327805074] [-9223372036854775808, 0, 6541869626013861625, -929030546, -6356085903210015655] [9223372036854775807, -8373154824159975475, 32679, 2180772025901512858, 502249748883523653, 9223372036854775807, -9223372036854775808, 245] [-1, 9223372036854775807, -1, -9223372036854775808, 9223372036854775807, -9223372036854775808, -9223372036854775808, 147483648] [32679, 32679, 9223372036854775807] [1, 0, -1, 0, -1] [-1, -1, -1, 0, 0, -993691371, -1, 4] [] [-2, 832130728, -1] [0.000000000000, -1.200000000000, 73.044700000000] [] [300.343000000000, 300.343000000000, 40.123000000000, 100.020000000000, 40.123000000000, 40.123000000000, 15.152700000000, 300.343000000000, 0.000000000000, 72.050600000000] [100.020000000000, 14.114100000000, 29.104700000000, 40.123000000000, 300.343000000000, 1.200000000000, 17.186300000000, 1.200000000000] [40.1230, 28.1784, 100.0200] [300.3430, 100.0200, 300.3430] [0.0000, -1.2000, 0.0000] [37.0260, 300.3430, 57.0946, 78.0964, 49.0220] [40, 0, 40] [300, -1, 83] [66, 300, 96, 100, 300, 76, 34, 100] [40, -1, 40, 0, 65, 1, 20, 31, 0, 1] ["--", "going", "don't", "-", "确认去掉系列", "国奥", "", "you're"] [] ["-", "", "when this", "", "--"] ["单条备份文件有车吗这种张先波哟偶那个新联惠不是", "?", "?"] ["-", "不曾", "there now not him I'll how all", "出团预算", "?"] ["", "自己摄像机可笑则会有不过呀", "--"] [] ["张海燕博威网通吸纳中兼容性钢琴曲用户认证条件青春", "?", "", "back why is are mean right you're", "", "天地世纪城历来科汇", "--", ""] ["w", "y", "o"] ["a", "u", "n"] [] [] [2024-01-08, 2025-06-18, 2014-08-12, 2025-06-18, 2023-12-09] [9999-12-31, 9999-12-31, 2024-01-08, 2025-02-17, 2024-02-18, 2023-12-20, 9999-12-31, 2024-06-30] [2023-12-12, 2024-06-30, 2025-06-18, 2024-01-31, 2023-12-12] [2023-12-14, 2023-12-17, 2023-12-12, 2025-06-18, 2025-02-18, 2023-12-11, 2024-02-18, 2024-01-09, 2024-01-17, 2024-01-19] \N \N [] [] [2025-02-18 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00] [2023-12-19 00:00:00, 2003-04-06 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00] [2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2014-08-12 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00] [2023-12-18 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-15 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123123] [2023-12-09 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00] [2025-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 9999-12-31 00:00:00] [2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00] +20 -346916527 -488520999 -223854435 32679 2023-12-12 2024-08-03 2024-01-19 2024-06-30 ? 零捌 she one he's was was in he's r 电子卡将刻骨铭心核好不好张婧俐设置计划考试饲料 just all back go back tell don't all 1 0 24.117800 300.343000 58.096400 40.123000 2024-06-30 12:01:02.123000 2024-02-18 00:00:00 2023-12-13 00:00:00 2025-06-18 00:00:00 2024-01-08 00:00:00 2024-01-31 00:00:00 2023-12-16 00:00:00 2025-06-18 00:00:00 [1, 0, 1] \N [1, 0, 1] [0, 0, 1] [-1, -33, 113, -118, -128] \N [1, 0, 0, 8, 2] [3, 57, -20, -1, 62, -128, 1, 98, 0, -128] [] [0, 10255, 245] [-30463, 32767, -1, 32767, -23685, 0, 0, -31558, 18769, 681] [245, -22155, -1, 245, -17115] [-2147483648, 32679, 2147483647, 946628029, -143882737, 661864660, 1, 1413279155, 2147483647, -460824572] [2043160590, 0, 147483648, 532770260, 2119347582] [] [1, -1429054465, 1732234812, -1621984321, 745685896] \N [-466013653165850291, -5218684931583798566, -1577389768, 0, -9223372036854775808, 7325484, -9223372036854775808, 1] [] [2180079064035506725, 7309551301316654871, 1, -9223372036854775808, -9223372036854775808, -6908076657407729774, 32679, -8812829290611932913] [] [-1, 0, 0, 1, 1961510222, 1, -1, 0, -574707769, -1] [-1, 0, 0] [1, -2, 0, -1890657042, 1, -2, -1, 0] [40.123000000000, 0.000000000000, 0.000000000000] [51.153800000000, 40.123000000000, 61.035100000000, 40.123000000000, 1.200000000000, 64.171400000000, 100.020000000000, 300.343000000000, 0.000000000000, 57.025200000000] [29.175000000000, -1.200000000000, -1.200000000000, 100.020000000000, 0.000000000000, -1.200000000000, 21.094200000000, -1.200000000000, 300.343000000000, 70.171600000000] [100.020000000000, -1.200000000000, 0.000000000000, 3.137000000000, 18.001900000000, 24.122100000000, 40.123000000000, 76.141700000000] \N [40.1230, 300.3430, 100.0200, 14.1329, 0.0000] [] [0.0000, 59.0811, 1.2000] [100, 0, 300] [-1, 100, -1, 40, 0] [300, 300, 0] [] ["可靠", "z", "博思元转到状态许愿树购买专区", "才是", "指纹", "加点意见免费见到引进过来是你或嵩县谈下来难道天马", "h", "?", "设置差点党经理这中间一部分换成", "do about were know"] ["she there had mean so as what", "on what he her as", "处理完", "饱和前台次要", "杭州"] ["you now some time his or just was here will", "be mean all", "", "所得税南阳倪玲梅看着补上叁佰汉威", "陈文杰"] [] ["had did get really then that's", "张贝", "f"] ["任性分数所作客服侠诺交到多点", "", "之道", "?", "?", "", "礼貌", "like", "不久特此证明授狗俊峰速达再发特此证明", "礼拜六翻译河南郑州等到"] ["神龙", "?", "--", "供货商", "沧桑"] ["now", "is okay have he i for right", "汉语南海", "--", "月底"] \N ["b", "r", "s"] ["j", "g", "n", "h", "f", "u", "a", "p"] ["y", "o", "q", "w", "i", "u", "k", "k"] [2023-12-13, 2023-12-17, 2023-12-15, 2023-01-15, 2014-08-12, 2026-01-18, 2024-02-18, 2025-06-18] [] [2023-01-15, 2024-08-03, 2023-12-15, 9999-12-31, 2023-12-17, 2023-12-14, 2024-02-18, 2023-01-15] [2023-12-09, 9999-12-31, 2023-12-14] [2025-06-18 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00] [2024-07-01 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] [] \N [2023-01-15 08:32:59.123000, 2023-12-14 00:00:00, 2023-12-19 00:00:00, 2023-12-15 00:00:00, 2024-01-19 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-31 00:00:00, 2023-12-09 00:00:00] [2023-12-09 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2017-02-08 00:00:00, 2025-02-17 00:00:00, 2023-12-16 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30] [2024-01-31 00:00:00, 2023-12-18 00:00:00, 2014-08-12 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123123, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [2024-01-19 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2024-01-09 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 2017-02-08 00:00:00] [2024-01-08 00:00:00, 2024-01-08 00:00:00, 2024-08-03 13:08:30, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00] +21 32679 946770243 1 -1 2025-02-18 2023-12-14 2024-07-01 2026-01-18 we say will a come he one but hey were 喝酒春江南阳倪玲梅离职调价高端年轻道理简历安排 技术员 don't u he's 做私单放风筝邮件关系区号 is 1 0 40.123000 1.200000 100.020000 1.200000 2024-01-19 00:00:00 2014-08-12 00:00:00 9999-12-31 23:59:59 2023-12-17 00:00:00 2025-02-18 00:00:00 2026-02-18 00:00:00 2024-01-09 00:00:00 2026-02-18 00:00:00 [1, 0, 1, 0, 1, 1, 0, 1, 1, 0] \N [0, 1, 1] [0, 1, 0, 1, 1, 0, 1, 0, 1, 1] \N [0, 77, 1, -63, -1, -1, 9, -8, 0, 1] [7, 0, 0, 127, -1, -45, 9, 7, -128, -65] [0, -1, -128, 1, 104, 2, -1, 1] [0, -32768, 10942, 30449, 0, -22360, -6147, 245, -4277, 6886] [-6083, -8891, 13831] [] [22576, -11998, 245, -27920, -23186] \N [650473225, 0, 1, 147483648, 923342515] [1183220867, 1757548554, 2147483647, -1, 1226290303, 2147483647, 865064938, 147483648] [-1, 0, -167071790, -1431284598, 1881345127] \N [727319833, -3921077, -8742044121798226593, -6931272232929803435, -9223372036854775808, -3012980, 32679, -1, 32679, -4317272550038049114] [-9223372036854775808, -8204311, -9223372036854775808] [0, -9223372036854775808, 147483648, -523523, 9223372036854775807] [0, 0, -1995130596] \N [] [-1, -1, -2, 9, -1, -2, 2, 0] [1.200000000000, 62.097300000000, 300.343000000000, 1.200000000000, 68.176800000000, 40.123000000000, 43.018700000000, 40.123000000000] [300.343000000000, -1.200000000000, 300.343000000000, 40.123000000000, 45.071100000000, 58.169700000000, 100.020000000000, 1.200000000000, 10.061200000000, 100.020000000000] [100.020000000000, 0.000000000000, 100.020000000000, 1.200000000000, 300.343000000000, 0.000000000000, 84.074600000000, 8.198900000000] [] [0.0000, 11.1275, 40.1230, 100.0200, 40.1230, 0.0000, 300.3430, -1.2000] [67.1703, 1.2000, 300.3430, 40.1230, 40.1230] [14.1052, 40.1230, 7.1419, 15.1322, 85.0469] [38.1955, 1.2000, 29.1866, 300.3430, 300.3430] [300, 34, -1, 46, 0] \N [] [] ["?", "培训莱克特天成苑烟厂统记表你杀时间系列", "联动", "平时王老师", "--"] ["提供", "-", "-", "单位", "-", "", "纽曼科技好好两台上网", "f"] ["结合谈好河南总代理机关号发给发布丰富电话本", "my that that's the look had on", "排列一开始神偷", "千兆网卡富绅查收钻石海天生命中飞信苦楚", "-", "her my with can't been could so", "理光四川笔记本正道专用宝莱占到疑问", "r", "", "time his yeah want didn't get this okay out"] ["", "i", "意思", "-", "some", "a", "say see then some", "who me him you're just", "-", "?"] \N ["", "--", "区别十五牡丹认可太厚坐坐国际企业立讯那是呀", "国企导演胡提", "词句"] ["难道", "--", "would my", "每次锐成烂漫正联具体你来办理生活周报顺河路王之", "--"] ["调试", "娃娃", "they", "?", "can't I'm no did", "-", "look was then he is go hey with", "?", "we it's you're yeah he been you're it", "-"] ["e", "z", "u", "x", "l", "y", "r", "i"] \N [] ["y", "u", "x", "j", "k", "i", "e", "v"] [] [2024-02-18, 2023-12-15, 2025-06-18, 2024-08-03, 2027-01-09, 2014-08-12, 2023-12-19, 2025-02-18] [2023-12-15, 2024-01-09, 2024-01-08, 2023-12-13, 2024-01-17, 2024-01-31, 2023-12-11, 2024-01-08] [2026-02-18, 2023-12-09, 2024-08-03, 2024-02-18, 2023-12-17, 2027-01-09, 2027-01-09, 2023-12-16] [2027-01-16 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00] [2027-01-16 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00] [2024-01-17 00:00:00, 2023-12-18 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00] [2023-12-12 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2023-12-19 00:00:00, 2027-01-16 00:00:00] [] [2027-01-16 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2023-12-11 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 9999-12-31 00:00:00] [2027-01-16 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00] [2023-12-14 00:00:00, 9999-12-31 23:59:59, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00] [2023-12-16 00:00:00, 2023-12-14 00:00:00, 2001-11-10 00:00:00, 2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2024-01-17 00:00:00, 2024-01-17 00:00:00, 2024-06-30 12:01:02.123000] \N [2014-12-25 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2024-01-17 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00] [2023-12-20 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00] +22 147483648 -393552232 32679 2147483647 2023-12-13 2025-06-18 2026-01-18 2026-02-18 - ? 依然 yes they time in we if in yeah see if get 中科 0 0 \N 300.343000 40.123000 100.020000 2027-01-09 00:00:00 2023-12-18 00:00:00 2024-01-09 00:00:00 2023-12-13 00:00:00 2024-01-17 00:00:00 2023-12-19 00:00:00 2024-02-18 00:00:00 9999-12-31 00:00:00 \N [0, 0, 0] [0, 1, 1, 1, 0, 0, 0, 1, 1, 0] [1, 1, 0] [127, 13, -128, 1, 4, 0, 0, -104, -19, 3] [-128, 127, -42, 127, -128, 27, 65, 0, 0, 60] [1, 0, 1, 0, 25, 0, -1, -63, -128, 1] [5, 1, 127, 38, -128, 34, -1, 3] [-1, 1, 25876] [-4058, -2933, -26681, 32767, -32768, 13682, 245, -16608] [-22149, 24577, 25889, -1, 6067, 10043, 25516, 12237] [-32768, -15676, -1, -14089, -1, -21415, 245, 6318] [1641758516, 1870806806, 2147483647] [1728223901, 1, 147483648, 1, -921411090, -2147483648, 60819825, -1] [2147483647, 465182221, -1079983337, -1, -125282788] [32679, 32679, -193594545, 1, -1052599531] [0, -554901636648052854, -4442097, 0, 1, -1, -1230539036, 2177851346503580890] [147483648, 1762162349458986511, -1, 245, -1, 1, -1, 245] [7020505678163679526, -9223372036854775808, -4461303008575250004, 9223372036854775807, -816008863582272173] [-1, 0, 258554517, 110630013, 32679] [0, 1, 612911484, 0, 527138171, -1, -1, 0] [-1, -2, 1, -1, -1, -1, 1, 2, 5, 1781081190] [7, -2, -1, 0, -1, -1, -1, 0] [] [82.101200000000, 40.123000000000, 40.123000000000] [0.026400000000, 8.104000000000, 85.085100000000, 56.130200000000, 100.020000000000, 29.026500000000, 33.108800000000, 100.020000000000, 48.067900000000, -1.200000000000] [0.000000000000, 96.074300000000, 0.000000000000, 63.050700000000, 1.200000000000, -1.200000000000, 67.158600000000, 40.123000000000] [23.040800000000, 0.000000000000, 53.082300000000] [0.0000, 1.2000, 100.0200, 9.1075, 63.1347] [-1.2000, 74.1819, 76.0525] [0.0000, 100.0200, 65.1276, 28.0229, 40.1230] [0.1846, -1.2000, 40.1230] [100, -1, 0] [300, 100, 100, 91, 40, -1, 40, 40] [88, 51, 50] [100, 79, 59] [] ["", "--", "got", "?", "爱过"] ["说过", "in who ok he's why mean have but who him", "?", "that's", "I'm"] ["she", "记下", "真实美女比人金达总经销商", "know from I'm the", "", "过节玖佰这样", "-", "at she on i at got got were right", "萍聚", "--"] ["?", "--", "go been at she we not have", "-", "a"] ["he's", "主机", "?", "look", "用不用写上", "佩服", "有车吗", "?", "--", "-"] ["", "to for like it's if here no he's you're up", "-"] ["h", "your well tell would got he mean from is say", "--", "-", "理解", "-", "送过去", "", "名称晚上聊", ""] ["z", "j", "m", "p", "w", "t", "c", "v"] [] ["o", "m", "y", "l", "o", "e", "x", "l", "a", "i"] ["p", "e", "d", "g", "f", "g", "x", "p"] [2024-01-09, 2024-01-31, 9999-12-31, 2023-12-15, 2024-01-09, 2024-01-08, 2023-12-10, 2025-02-18] [9999-12-31, 2023-12-16, 2024-01-09, 2023-12-11, 2024-08-03, 2006-11-07, 2024-01-31, 2023-01-15] [2023-12-10, 2024-01-31, 2025-06-18] [2024-02-18, 2024-02-18, 2024-02-18, 2024-08-03, 2023-12-13, 2024-02-18, 2024-06-30, 2018-11-22] \N [9999-12-31 23:59:59, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [2023-12-16 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59] [2023-12-11 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00] [2026-01-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-01-31 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00] [2023-12-09 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2023-12-13 00:00:00] [2023-12-20 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [] [2023-12-14 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00] [2027-01-09 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02.123000, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00] [] [2023-12-17 00:00:00, 2023-12-12 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2023-12-12 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30] +23 -1851896740 -2147483648 32679 147483648 2025-02-18 2023-12-13 2023-12-17 2014-08-12 -- about 下载 我能 长信 ? 都有百圆磁盘华康数字显示网站性质做到听着容易 1 0 6.047400 34.072500 66.159300 300.343000 2023-12-14 00:00:00 2025-02-18 00:00:00 2024-06-30 12:01:02.123000 2023-12-13 00:00:00 2023-12-11 00:00:00 2024-01-09 00:00:00 2023-12-12 00:00:00 2023-12-20 00:00:00 [1, 0, 1, 1, 0] [0, 0, 0, 0, 0, 0, 1, 0] [1, 0, 1, 1, 0, 1, 1, 0, 0, 1] [1, 0, 0, 0, 0] [87, 6, 115, 1, 58, 93, 127, -69, 127, 1] [-1, -128, -128, -8, 42, -128, -128, 57] [127, -69, 0] [1, 1, 127] [-16985, -26579, 1, 32767, -26434] [2470, -32409, -20374, -6923, 32767, -27430, -1, -4735] [-9635, 18799, 1, 24668, 16462, 7254, -31111, 0] [-18874, -1, 0, 32767, 28499, -15837, 32767, 245, -19584, -7935] [-1041850185, 209868378, -1988548732] [32679, -1873840247, 0] [1400771749, -1, 0] [1502187432, 147483648, 180538386, -1248773780, -1, 32679, -1, 495482516, -597059407, -1] [] [147483648, -9223372036854775808, -841624210523677557, 9223372036854775807, 245, 0, -1613693172, 32679, 0, 245] [4639765925252197452, 9223372036854775807, -6486788, -6783511002680104232, 32679, -3662388413450398949, 32679, 9223372036854775807, 0, 1436538374] [147483648, 245, -1] [-2, 1, -2, 0, 9] [-2, -2, -1, -1, 2] [0, 0, 1, -1, -2, 2092053911, 1, -2, -1, 0] [0, 1, -1, 8, -2] [40.123000000000, 0.000000000000, 40.123000000000, 98.079900000000, 75.153500000000, -1.200000000000, 2.010700000000, 100.020000000000] [1.200000000000, 12.194400000000, 64.129900000000] [0.000000000000, 0.000000000000, -1.200000000000, 0.000000000000, 40.123000000000, 40.123000000000, 100.020000000000, -1.200000000000, -1.200000000000, 35.111800000000] [100.020000000000, 71.098600000000, 1.200000000000] [100.0200, 40.1230, 55.0648, 39.0327, 100.0200, 100.0200, 43.0802, 100.0200] [300.3430, 62.1706, 13.0258, 100.0200, 300.3430] [] [40.1230, 0.0000, 39.0818, 100.0200, 100.0200] [] \N [54, 100, 37, 44, 0, 100, 100, 100] [1, 100, 300, 30, 1, 100, 100, 100] ["网通腾飞", "相处经销商南村", "me do do about to why i see you", "-", "", "彼此内训品牌", "周星驰", "was", "well", "get good oh look me come his are and why"] ["-", "n", "come"] ["", "", "say had something got no", "前几天城市", "聊过"] [] [] ["?", "t", "还有", "", "奋斗机柜"] ["百货", "like", "--", "of you like go back be no or see", "今年文宁港湾在他家架子小雪转账不想发邮箱问道"] ["?", "舒展", "创恒电子", "始终", "--", "清单", "?", "-"] ["a", "m", "v", "l", "p", "n", "a", "e"] \N ["i", "e", "w"] ["z", "f", "v"] [2025-06-18, 2025-06-18, 2023-12-11, 2024-07-01, 2000-01-18, 2024-06-30, 2026-01-18, 2024-02-18] [2014-08-12, 2024-02-18, 2024-01-09, 2023-12-16, 2025-02-18, 2023-12-17, 2026-01-18, 2024-01-08, 2023-12-12, 2024-07-01] [2023-12-11, 2023-01-15, 2024-08-03] [2023-12-10, 9999-12-31, 2024-06-30] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2024-08-03 13:08:30] [2026-02-18 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00] [2008-05-17 00:00:00, 2024-06-30 12:01:02, 2024-07-01 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00] [2023-12-17 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2014-08-12 00:00:00, 2024-01-19 00:00:00] [2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] [2024-07-01 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00] [2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00] [] [2024-07-01 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00] [2023-12-09 00:00:00, 2000-03-26 00:00:00, 9999-12-31 00:00:00] [] [2023-12-10 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] +24 2147483647 -1098616283 -2147483648 -1280463105 2023-12-10 2025-06-18 2023-12-19 2023-12-16 那种不设置不好五万百家麦迪官方 because 看不到 who see I'll time with did could can't didn't a ? 0 0 1.200000 36.189400 5.021600 300.343000 2023-12-13 00:00:00 2023-01-15 08:32:59.123000 2025-02-17 00:00:00 2027-01-16 00:00:00 2023-12-19 00:00:00 2023-12-16 00:00:00 2023-12-18 00:00:00 2014-08-12 00:00:00 [] [0, 0, 1, 0, 1, 1, 1, 0] [1, 1, 0] [1, 0, 1] [51, 0, -18, 75, 1] [] [110, -1, 1, 0, 1, 0, -11, 0] [89, -1, 0, 127, 0, 0, 0, -1, -1, 1] [12142, 245, 13798, 5981, 0, -10059, 245, 14903] [5629, 32767, -32768, 13333, 245, -32768, -32768, 245] [26292, 245, -1] [-16671, -13911, 24192] [-599060787, -1322007385, 32679, -1, 1850836226, -596833256, 1, -2144143491] [-1051255901, 1, 0, 1, 32679, -2147483648, -1426341445, -481771632] [32679, -373801479, 505494243, -1, -1, 1, 1232550057, 32679] [-1314297091, -1863137184, 0, 2026419384, -1, 32679, -2147483648, 2147483647] [1953561009122699171, 147483648, -1] [] [] [0, 2972426456084555123, 6566598210641915505, -1, 0] [1, -1, -1206849312, 0, 1, -2, 0, 2, 6, 0] [9, -1, -1, -2013813682, -1, 0, 3, 1, -485418178, 4] [2, 1, -2, -2, -1, -1, -1, 1, -1, 2] [2, -1, -822061478, 1, 1, -2, 1799535731, -1] [71.032100000000, 0.000000000000, 100.020000000000, 100.020000000000, 40.123000000000, 41.116000000000, 60.182400000000, 100.020000000000] [1.200000000000, 49.036200000000, -1.200000000000] [] [-1.200000000000, 2.180700000000, 5.109300000000] [40.1230, -1.2000, 300.3430, 70.0849, 0.0000, 1.2000, 0.0000, 300.3430] \N [40.1230, 40.1230, 300.3430] [] [85, 66, 38] [85, 1, -1, 1, 1] [-1, 0, 97] [300, 0, -1] ["几篇", "", "say", "合兴小心情况下", "we as can't that's know", "交通厅", "had his", "mean"] \N [] ["?", "on", "?", "a", "-"] [] ["中层经理", "will it she he were i don't know", "will been right say good time", "高杰", "tell"] ["柒服装金色速达你问讨论组等于你怎么去的微波专门不下", "--", "?", "--", "", "right here we have hey be like do of on", "?", "?", "几下领出看完", "另外按季度服务"] ["联系", "", "s", "你早几天回来吧", "库里有钱老公收藏夹动手脚制动阀烟火"] ["c", "o", "b"] ["u", "w", "c", "u", "l"] ["t", "k", "r", "c", "c", "f", "v", "i"] [] [] [9999-12-31, 2023-12-20, 2024-01-09] [2014-08-12, 2023-12-20, 2023-12-20, 2024-01-08, 2025-06-18, 2023-12-17, 9999-12-31, 2027-01-16, 2023-01-15, 2026-02-18] [2024-02-18, 2023-01-15, 2024-01-31, 2023-12-19, 2023-12-20] [2024-01-08 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00] [2023-12-10 00:00:00, 2014-08-12 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00, 2024-06-30 12:01:02, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00] [2023-12-16 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00] [2023-12-14 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00] [] [9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123000, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00] [2024-01-31 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2014-08-12 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 9999-12-31 23:59:59, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00] \N [] [] [2026-02-18 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00] +26 -2147483648 -1862307983 -646571091 32679 2026-01-18 2025-02-18 2023-12-16 2024-08-03 -- ? \N o 短语 s y - 0 1 40.123000 53.109800 -1.200000 -1.200000 2024-02-18 00:00:00 2024-06-30 12:01:02.123000 2024-01-09 00:00:00 2023-01-15 08:32:59.123000 2000-01-05 00:00:00 2023-12-13 00:00:00 2023-12-15 00:00:00 2023-01-15 08:32:59.123123 [0, 0, 0, 0, 1, 0, 1, 0, 1, 1] [1, 0, 0, 0, 0, 1, 1, 1, 1, 0] [1, 1, 0, 1, 0, 1, 0, 1] [0, 0, 1] [-96, -128, 1] [-1, 7, -128, 127, 76, 9, 106, -128, -1, -118] [9, 47, 127, 0, 0] [127, -128, 127] [-1, -5119, -2239, -8362, -12150] \N [] [] \N [-233967088, 147483648, -663871800, -133582737, 0] [] [0, 0, 32679, 147483648, -1] [32679, 9223372036854775807, 1320560906339036018, -553264258, 245, -1, -1, 32679] [-8832455233793110946, 777053, -4936091590673625099] [147483648, -9114044455763085760, 527225472012032384] [] [-2, 1, 560078147, 5, 0] [-2, 372545549, 1] [8, -2, 8, -1, 0, 5, -1, -1, -2, 4] [2, -1527685257, -2, -2, -1] [40.123000000000, 77.061000000000, 11.027400000000, 1.200000000000, 0.000000000000] \N [-1.200000000000, 57.085100000000, 1.200000000000, 40.123000000000, -1.200000000000, 1.200000000000, 100.020000000000, -1.200000000000] [1.200000000000, 1.200000000000, 94.136000000000, 40.123000000000, 18.136700000000] \N [0.0000, 50.0183, 66.0603, 100.0200, 3.0824] [300.3430, 0.0000, 0.0000] [86.1443, 300.3430, 1.2000, 40.1230, 300.3430, 1.2000, 300.3430, 100.0200, 1.2000, 100.0200] [0, 38, 27, 0, 300, -1, 40, 0] [1, -1, 19, 0, 0, 9, 93, 1] [92, 83, 2, 19, 300] [1, -1, 300, -1, 300, 40, -1, 16] ["out", "ok", "诺顿赵总新买数目", "", "e", "小心小浪底不含反对针对吃饭哪许愿树留个", "", "万一", "can't you", "don't see tell he"] ["", "", "i", "解难大话几个新山大姐阿奎那", "开除"] ["达成率", "如果", "did how"] ["?", "your good", "tell", "be", "-"] ["--", "this have he's out would no of he's", "yeah", "了解失陪医生英莲事业经济金总孙海洋", "客运量", "出面哟偶那个自己启动一定吴静线缆下个战友", "one", "of"] ["--", "all", "but"] ["?", "在欧普尾货贰台实际苦楚发邮箱谁知道不太交叉口经理", "", "r", "这家"] ["g", "tell", "智博", "back hey is look but here then out", "减去工商局报账", "this he's really right", "", "he's", "新拓", "?"] \N [] ["i", "a", "o", "b", "d"] ["j", "n", "m", "s", "y", "e", "o", "z", "e", "i"] [2025-02-18, 2024-01-08, 2023-01-15, 2027-01-16, 9999-12-31] [2025-02-17, 2005-01-21, 2023-12-13, 9999-12-31, 2024-01-19, 2024-01-17, 2023-01-15, 2023-12-19] [2024-01-19, 2023-12-19, 2027-01-09, 2024-01-17, 2023-01-15, 2024-01-17, 2024-01-19, 2023-12-17] [] [2025-02-17 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59, 2027-01-09 00:00:00] [2024-01-17 00:00:00, 2023-12-11 00:00:00, 2015-06-15 00:00:00, 2024-06-30 12:01:02, 2023-12-19 00:00:00] [2023-12-15 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-19 00:00:00] [2025-02-17 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00] [2024-07-01 00:00:00, 2023-01-15 08:32:59.123000, 2027-01-09 00:00:00, 2016-06-06 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00] [2023-12-11 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2004-01-14 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00] [] [] [9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2024-01-19 00:00:00, 2023-12-11 00:00:00] [2023-12-19 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00] +27 -1 \N -1380212922 147483648 2025-06-18 2025-06-18 2024-01-09 2023-12-11 \N \N 热销真理抓住 具体你来办理五洲迅盘近台南划分公道话 -- can't no some if she but 1 1 0.000000 76.144800 1.200000 100.020000 2024-02-18 00:00:00 2014-02-02 00:00:00 2023-12-15 00:00:00 2023-12-11 00:00:00 2023-12-16 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2025-06-18 00:00:00 \N [1, 0, 0, 1, 0, 1, 0, 1, 0, 1] [1, 0, 1, 0, 1, 1, 1, 0, 0, 0] [0, 0, 1, 0, 0, 0, 0, 1, 1, 1] \N [51, 0, -37, 127, -1, -128, 21, 127, 1, -126] [1, 1, 3] [5, -124, 0, 6, -128, 17, 8, -94] [-32768, 1, 0, -1, -1] \N [] [-16649, 16824, -32768, 32767, 2809, 0, -32768, 7008] [] [-597920522, 723630121, 147483648, 0, -906479150, 32679, 721788443, 2085993937, 468380784, 32679] [] [-2147483648, 0, -1, -2147483648, 2147483647, -1237046477, -1, 2147483647] [147483648, -7352023829659788986, 5171590, -9223372036854775808, 1424382473] [-1391384444717924229, 245, -1851499864854549724, 32679, 32679, 341746221862947483, 32679, 32679] [-8372854200519706255, 2457099575282859519, -9223372036854775808, 0, 5380655, 2664762892677775175, 245, 2991130570226682818, -8567901472941147595, -6229844401850770765] [-1, -1, 245] [1, 1791990047, 2095413315] [-336171888, 0, 5] [0, 0, -360735166, 1113539223, 0, -1, 1, -2] [-787173066, 6, -1, 1, 1, -1, -2, -2] [100.020000000000, 100.020000000000, 0.000000000000, -1.200000000000, 0.000000000000, 79.163000000000, 51.081400000000, 300.343000000000] [40.123000000000, 100.020000000000, 11.158000000000, -1.200000000000, 89.169300000000, 31.071100000000, 0.000000000000, 1.200000000000] [1.200000000000, 0.000000000000, 40.123000000000, 12.115500000000, 300.343000000000, 73.112500000000, 30.093700000000, 300.343000000000, -1.200000000000, 40.123000000000] [] [] [1.2000, 6.0129, 1.2000, 83.0813, 2.0611, 0.0000, 41.0735, 1.2000] [300.3430, 100.0200, 44.1232] [-1.2000, 50.1801, 5.0867, -1.2000, 0.0000, 300.3430, 4.1590, 40.1230] [53, 1, 100, 62, 40, -1, 19, 60] [100, 300, 0, 69, 300, 40, 1, 40, 100, 100] [40, 1, 40, 1, -1, 100, -1, 43] [100, 12, 40, 62, 0, 69, 1, 0, 40, 91] ["g", "意识重复盛唐在欧普更好漯河信息化天数饱和", "well his out all think know get what", "?", ""] ["李金才那六无论单身", "?", "艾蕊"] ["see", "?", "I'll hey because have they", "could", "p", "-", "惬意", "j"] ["还在吗减去学校叶子", "do and if she right know get one oh that", "l", "I'm", "站在", "出面不含税", "ok", "-", "改动上面两台正规魏军难道回来陇海还有", "--"] ["是吗", "be", "水平", "--", "r", "?", "want", "m"] ["-", "", "郑东", "", "金冠", "", "最新版本", "say"] ["", "", "that's", "顶多", "got", "b", "who he's not", "get"] ["表示申请到货请客内部三门峡磁带机给我带来", "-", "理想", "否则", "你好到底领先广角国标肆仟黄经理"] ["d", "z", "s", "x", "p", "r", "v", "a"] \N ["g", "e", "w"] [] [2024-08-03, 2023-12-11, 2023-12-18, 2023-12-09, 9999-12-31, 2025-02-17, 2023-12-15, 2023-01-15, 2024-01-17, 2003-09-15] [2023-12-10, 2025-06-18, 2026-02-18, 2023-12-20, 2024-08-03, 2024-08-03, 2024-02-18, 2024-01-19] [2014-08-12, 2024-02-18, 2016-05-26, 2023-12-14, 2024-02-18, 2023-12-14, 9999-12-31, 2024-06-30, 2026-02-18, 2024-01-08] [2027-01-09, 2023-12-14, 2014-08-12, 2024-01-31, 9999-12-31, 2024-06-30, 2026-02-18, 2024-07-01] \N [2023-01-15 08:32:59, 2023-12-18 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59, 2025-02-18 00:00:00, 2023-12-14 00:00:00] [2024-02-18 00:00:00, 2011-12-15 00:00:00, 2023-12-18 00:00:00] [] [2023-12-19 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00, 2023-12-15 00:00:00] [] [2024-01-08 00:00:00, 2023-12-20 00:00:00, 2000-12-22 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00] [9999-12-31 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2018-04-18 00:00:00, 2027-01-16 00:00:00] [2025-06-18 00:00:00, 2023-12-18 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2014-08-02 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30] [2024-01-19 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2014-08-12 00:00:00, 2014-08-12 00:00:00] [2026-01-18 00:00:00, 2023-12-14 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2011-04-06 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00] +28 32679 147483648 -1 147483648 2023-12-20 2023-12-15 9999-12-31 2023-12-19 mean -- 不对战友 don't -- 不定一栏很想聊天备注按季度返点丹尼斯 -- 0 0 0.000000 -1.200000 100.020000 300.343000 2023-01-15 08:32:59.123000 2023-12-09 00:00:00 2023-12-17 00:00:00 2023-12-20 00:00:00 2024-07-01 00:00:00 2025-02-18 00:00:00 2024-01-08 00:00:00 2023-12-20 00:00:00 [0, 0, 1, 0, 1, 1, 1, 0] [1, 0, 1] [1, 1, 0, 0, 0] [0, 1, 1, 1, 1, 1, 0, 0, 1, 1] \N [98, 127, 23, -1, 121, 1, 42, -1] [0, 1, 55, 1, 0, 0, 1, 78] [-74, 5, 41, -128, -63, 0, -30, 4, -8, -1] [32767, 15644, 22126, 3723, -2447, 245, 0, 19007] [-32768, -1407, 0, 20523, 0, -32768, -31388, 6304] [245, 245, 245, 245, 28351] [32767, 10367, -32768] [-597168472, -1945770914, -1, 0, -362112291, -1582323460, 32679, -1, -1, 0] [-2147483648, -2147483648, 147483648, 0, 2147483647, -651593347, 854405437, 2115186622] [-1426833135, -2147483648, 32679] [] [-1, 0, 0, 9223372036854775807, -1, -8459729693801352086, 245, -1] [-1, 1241429275, 2873126156489586535, 7907056054429183499, -9223372036854775808, 7878997566851077222, 245, 9223372036854775807, 32679, 245] [245, 9223372036854775807, 6258505036791720834, 1, 32679, 245, -9223372036854775808, 1] [245, -1, 32679, 1, -2544523] [6, 0, 4, -1, 1, -2, 1, -1930813897, 0, -1167524744] [1, 1743167484, -2, -1, 0] [] [0, 1, -2, -2, 4, -869797002, -2, 1] [-1.200000000000, -1.200000000000, 1.200000000000, 62.167600000000, 40.123000000000, 100.020000000000, -1.200000000000, 48.095300000000] [41.185500000000, 35.082000000000, 37.000800000000] [] [300.343000000000, 100.020000000000, 0.000100000000, -1.200000000000, 90.171400000000] [] [0.0000, 1.2000, 0.0000, -1.2000, 40.1230, 37.1341, 0.0000, 15.0553] [] [0.0000, 1.2000, 0.0000, 0.0000, 40.1230, 0.0000, 0.0000, 40.1230] [66, 300, 20, 100, -1, 1, 0, 100] [1, 20, 100, 1, 40, 24, 81, 1] [35, 300, 78, 40, 300] [40, 100, 40] ["汉威", "--", "国企吃饭哪它是需要帮忙治疗折旧老乡意见数据库", "we", "语文", "元整单个实力那边", "有人", "?", "自学", "they yeah yeah"] ["-", "at up just", "--"] ["--", "b", "can't they they like about", "上演报价这么做热销中小孩玖佰向你最快双千兆款且", "发到", "well", "now", "--"] ["why", "on", "", "all", "your", "then have or what really", "--", "", "v", "-"] ["s", "-", "就有", "", "色戒", "just", "改掉", "?"] ["不低于", "海域哦", "", "as", "get now", "me a I'm time the okay I'll that's", "", "-"] ["okay there out get go could going the then did", "--", "不应该", "don't", "五千"] ["人往高处", "", "how", "we then to she tell it", "-", "感悟弄个折叠行政", "no when right", "周岁"] [] ["h", "h", "s", "p", "p", "d", "j", "a", "e", "e"] ["r", "e", "g", "d", "m"] ["h", "a", "l", "s", "z", "e", "v", "h"] [9999-12-31, 2023-12-11, 2024-06-30, 2014-02-24, 2026-02-18, 2024-02-18, 2023-12-20, 2014-08-12, 2023-12-17, 2026-02-18] [2023-12-15, 2025-02-17, 2025-06-18, 2026-01-18, 2023-12-18, 2014-08-12, 2014-08-12, 2025-02-18, 2024-01-08, 2024-01-08] [2024-08-03, 2025-02-18, 2025-02-17, 2025-02-17, 2024-02-18] [2024-01-31, 2024-01-31, 2023-12-13] [2027-01-09 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2023-12-12 00:00:00, 2024-01-19 00:00:00, 2023-01-15 08:32:59, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00] [2023-12-17 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00] [2027-01-09 00:00:00, 9999-12-31 00:00:00, 2014-08-12 00:00:00, 2003-03-16 00:00:00, 2023-01-15 08:32:59] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00] [2025-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30] [2017-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 2008-09-16 00:00:00, 2014-08-12 00:00:00] [2023-12-17 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00] [9999-12-31 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [2024-08-03 13:08:30, 2027-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-17 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00] [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00] +31 32679 0 -53347858 625968741 2024-08-03 2023-12-18 2024-06-30 2025-06-18 tell did they he who 快运刘雪 - why about okay that he's at for now one 吸纳中 什么的招待咱们 1 1 40.123000 1.200000 65.172200 0.000000 2024-01-09 00:00:00 2026-01-18 00:00:00 2026-01-18 00:00:00 2024-02-18 00:00:00 2023-12-11 00:00:00 2024-01-17 00:00:00 2023-12-19 00:00:00 2023-12-17 00:00:00 [0, 0, 1, 0, 0, 0, 0, 0, 0, 0] [0, 1, 1, 0, 1, 1, 1, 0, 1, 1] [] [1, 0, 0] [-1, 127, -128] \N [103, -1, -128] [] \N [-19901, 10374, -1] [-1, -6500, 32467, 32041, 245, 15662, -20676, -17563] [0, 245, -17970] [1, -1659140410, 147483648] \N [] [] [-1, 9223372036854775807, -1434558303, -9223372036854775808, 245, 32679, 595610362, 32679, 245, 9223372036854775807] [] [32679, 0, 287109666150754658] [] [4, 1, 1] [1, 5, -201861276, 1638664111, 1, -766197217, 0, -1, 1524576969, -2140699671] [] [] [73.153800000000, 40.123000000000, 53.077800000000] [300.343000000000, 0.000000000000, 66.027800000000] [-1.200000000000, 52.122700000000, 100.020000000000, 12.000100000000, 94.074200000000, 100.020000000000, 21.079600000000, 1.200000000000] [11.074400000000, 100.020000000000, 0.000000000000, 1.200000000000, 20.153700000000, -1.200000000000, 300.343000000000, 1.200000000000] [100.0200, -1.2000, 100.0200, 58.1885, 87.1069, 1.2000, 40.1230, 0.0000] [1.2000, 80.1434, 300.3430, 53.0111, 300.3430] [] [300.3430, 26.0096, 100.0200, 38.0442, 7.0775, 73.0958, -1.2000, 100.0200] [100, 84, 100] [83, 100, 100, 4, 88, 0, 1, 60, 14, 1] [0, 58, 40, 0, 76, 0, -1, 0] [0, 40, 39, 300, 5, 1, 300, 0] ["针对天星丁丁大风能不鞥造成在职弯弯的月亮一万晓燕", "--", "电话英文广角房子", "o", "do it's as", "不能说出点加为好友国泰", "", "-", "--", "?"] ["", "", "-", "good it's yeah can", "意思", "?", "陇海", "ok", "耐心而", "?"] ["百脑汇加密狗没有货跟我跟着情人节也在一想单个文利", "are", "合作单位进一步", "got", "贰仟现金结权利地区呀库存有意向", "-", "冤枉新版指出来", ""] ["", "--", "早着呢", "?", "know", "短语", "", "from", "付出了", "--"] ["have", "-", "充满", "也有当地恶心死全系列硬盘下了安徽", "as that's he the just we because could are"] ["柏霖", "-", "-", "", "?", "?", "x", "something this", "say there if a good think", "走过去"] [] ["-", "夏娜", "泰安", "we had", "is"] ["q", "h", "i", "c", "b", "r", "y", "y", "g", "j"] ["d", "x", "v"] [] ["r", "y", "z", "m", "w", "x", "x", "a"] [2023-12-10, 2023-12-10, 2023-12-11, 2024-01-09, 2024-01-08, 9999-12-31, 2024-01-19, 2024-07-01, 2026-02-18, 9999-12-31] \N [2027-01-16, 2023-12-12, 2023-12-19, 2023-12-13, 2023-12-16, 2025-02-17, 2024-07-01, 9999-12-31, 2024-07-01, 2025-06-18] [2025-02-17, 2023-12-10, 2024-01-17, 2023-12-10, 2026-01-18, 2023-12-11, 2023-12-10, 2023-12-11] [2024-01-17 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-15 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59] [2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00] [2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2016-06-14 00:00:00, 2024-01-17 00:00:00, 2024-08-03 13:08:30, 2025-02-17 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2024-01-19 00:00:00] [2023-12-16 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2015-05-14 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00] [2023-01-15 08:32:59.123000, 2023-12-11 00:00:00, 2023-12-14 00:00:00] [] [2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-31 00:00:00, 2024-01-17 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00] [2023-12-15 00:00:00, 2024-01-19 00:00:00, 2016-10-13 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00] +32 \N \N 2147483647 1003742728 2024-06-30 2024-01-31 2025-02-17 2025-06-18 well know ? 广告词 going do v 创新祝福陈文杰周哥过节王子做不到会想间进来慧眼 1 0 100.020000 3.131800 300.343000 0.000000 2023-12-16 00:00:00 2025-02-17 00:00:00 2027-01-16 00:00:00 2023-12-11 00:00:00 2024-02-18 00:00:00 2025-02-18 00:00:00 2023-12-16 00:00:00 2025-02-18 00:00:00 [1, 0, 1, 0, 0, 0, 1, 1] [1, 1, 1, 0, 1, 1, 1, 0] [] [0, 1, 1, 1, 1] [] \N [7, 58, 117] [-94, 127, 127, -82, 0, -128, 2, 4] [-1, 26391, -15125, 0, 0] [1, -1, 20669, 1, 21776, 32153, 7026, -1, 18324, 32767] [18904, 1447, -9698, 21030, 22779, 32767, 6390, 0] [257, 32767, 15633] \N [-2075928750, 0, 1, -1216218549, 1] [-2117495067, 2147483647, 1610076315] [] [249823467, 1, -1790406910, -9223372036854775808, -9223372036854775808, 147483648, 0, 9223372036854775807] \N [1, 245, -1] [-9223372036854775808, 3717874, 9223372036854775807, -2661235, 0, 3054261877902090805, -9223372036854775808, 6867031559451882106, 245, 0] [0, -485980272, -2, 1782728425, -1, 0, 955971981, -1, 1, 1] [] [1, -1943387993, -1, 0, 2029993265, -2, -1, -2] [0, 1353984857, 0, 1, -2] [0.000000000000, -1.200000000000, 85.162800000000] [-1.200000000000, 40.123000000000, 0.000000000000] [0.000000000000, -1.200000000000, -1.200000000000, 100.020000000000, 78.174700000000, 100.020000000000, 1.200000000000, 1.200000000000] [40.123000000000, 40.123000000000, 300.343000000000, 40.123000000000, 80.082200000000] [84.1991, 80.0838, 0.0000, 0.0000, 100.0200, 0.0000, 40.1230, 1.2000] [100.0200, 0.0000, 40.1230] [0.0000, 100.0200, 68.1862, 0.0000, 7.1890, 42.0921, 40.1230, 100.0200] [] [1, -1, 1, 54, 1, 300, 1, 45] [-1, -1, -1, 100, 40, 300, -1, 1, 0, 300] [13, 96, 54, 38, 300, 300, 0, 95, 300, 100] [40, 18, 100, 39, 0, 44, 300, 1] \N ["巴巴全部前端懂吗热情劲", "t", "--", "", "with a all the had don't have on", "going", "?", "why", "?", "i"] ["面子图形工作站中盈盈规定是不是你能看到濮阳原来屠龙记", "封库", "-", "for", "在家", "投影机就要独显不问", "not", "--", "for had when back", "研究院"] ["well", "had if with get to i so yeah what why", "--"] ["see", "--", "-", "v", "几万", "why", "爱人", "get do this all", "?", "--"] ["no just have on your on in be yeah", "his", "鑫辉", "--", "--", "will", "--", "time"] ["I'll", "we", "there"] ["小姑娘", "a", "?", "", "九九这样伴奏刘亮正事看来忙碌", "货运", "your didn't is I'll now but like something", "x"] ["m", "j", "j", "n", "a", "j", "q", "p"] ["z", "w", "q", "z", "g", "f", "e", "h"] ["k", "k", "t", "r", "y", "z", "l", "u", "k", "d"] ["d", "q", "i"] [2014-08-12, 2023-01-15, 2024-01-08, 2023-12-14, 2023-12-10, 2023-12-14, 2025-02-17, 2023-12-11, 2023-12-12, 2025-06-18] [2026-02-18, 9999-12-31, 2025-06-18, 2024-01-08, 2024-06-30, 2023-12-19, 2024-02-18, 2024-01-09, 2024-01-08, 2025-06-18] [2027-01-09, 2023-12-11, 2023-12-15, 2023-12-19, 2024-07-01, 2027-01-16, 2023-12-13, 2023-12-09, 2026-02-18, 2023-12-17] [2024-01-09, 2023-01-15, 2024-02-18, 2023-12-15, 2026-02-18, 9999-12-31, 2024-02-18, 2024-02-18] [] [2024-02-18 00:00:00, 2023-01-15 08:32:59, 2024-06-30 12:01:02, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00] [] [2024-01-08 00:00:00, 2023-12-19 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00] [2025-06-18 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00] [] [] [2024-07-01 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2023-12-13 00:00:00, 9999-12-31 23:59:59] [2023-12-11 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-16 00:00:00] [] [2024-06-30 12:01:02.123000, 2023-12-12 00:00:00, 2024-02-18 00:00:00] +33 \N -370135458 2147483647 1 2023-12-14 2027-01-09 2026-02-18 2023-12-16 进账 -- y 农业路 治好 郑州市 now ? \N 0 1.200000 40.123000 53.180000 -1.200000 2026-01-18 00:00:00 \N 2024-01-19 00:00:00 2024-02-18 00:00:00 2026-02-18 00:00:00 2005-04-21 00:00:00 2023-12-09 00:00:00 9999-12-31 00:00:00 [1, 0, 1, 1, 0, 0, 0, 0, 0, 0] \N [0, 0, 0] [1, 0, 0, 0, 1, 1, 0, 0] [1, -115, -101, -122, -1, 127, -123, 22] [-86, -128, -128, 81, -11, 0, -1, -128, 1, -128] [-128, 127, 127, 7, -128] [-22, -68, 1, -20, 127, 8, 5, -128, 1, 1] [245, 0, 1, -22794, 1, 15090, 23339, -32768, 31608, 1] [-32768, 245, -9486, 1, 32060] [] [245, 30019, -1, 32767, 9494] \N [] [] [-1961849674, 193966664, 422431190, 1, -468554165, 699212471, 404229755, 32679] [32679, 0, 0, 3286214370517988451, 32679, 245, 4197877529599283055, 147483648, 32679, 147483648] [0, -3798398123083331678, 245] [9223372036854775807, 2665488150706425999, 32679] [925820922, -9223372036854775808, 245, -9223372036854775808, -2478979983082622031, 0, 0, -1499713323] \N [0, 1, -1, -1, -2, 9, -1, -1] [-845664459, -1, 5, 319842219, -1] [8, 3, 1, 516597694, -2, 1570470416, 0, -2] [96.089000000000, 55.037400000000, 0.000000000000, 82.152300000000, 0.000000000000] \N [34.196900000000, -1.200000000000, 73.059700000000] [42.087200000000, 40.123000000000, 61.035900000000, 31.058200000000, 53.179900000000] [40.1230, 0.0000, 100.0200, 0.0000, 40.1230] [] [] [0.0976, -1.2000, 30.0470, 300.3430, 33.1093, 100.0200, 300.3430, 100.0200] [1, 300, 85, 300, 100, 40, -1, 0] [-1, -1, 14, 78, 40, 1, 71, 40] [300, 40, 76, 0, -1] [] ["--", "?", "经三路意思节日", "虽然做梦乔总客运量早晨没沈晓海", "-", "讲的江河企业级领导麻烦死回来年华炒货限价问好", "各位认为", "?"] ["about it's ok right my they", "", "him", "oh say yes your i that's because because", "at her one as", "yes", "设和宏运注册", "-", "-", "then yeah here this then was like a have"] ["going", "just", "how", "--", "get if just"] [] ["or", "看是", "", "see", "they it some were her are", "--", "东风系列", "-"] ["can't", "为你", "看法工具如此", "公路", "think that's come no in as something i time", "地区", "will", "if"] ["感情还行数据哦不对外词库陈老师建新最大化谢谢安排", "", "-", "最深刻列表风格好人一下", "答案", "才回来泡茶麦迪在他家", "凯峰剑派书生所要求封库", "问了", "", "我一会给大"] [] \N ["v", "h", "w"] ["m", "w", "u", "r", "g", "s", "y", "g", "t", "j"] ["w", "q", "x", "y", "w", "o", "t", "s"] [2025-06-18, 2023-01-15, 2023-12-14, 2024-08-03, 2026-01-18, 2024-01-19, 2024-01-17, 2025-06-18, 2025-02-17, 2026-01-18] [2025-02-17, 2023-12-13, 2027-01-09, 2025-06-18, 2023-12-18, 2023-12-14, 2024-02-18, 2023-01-15, 2024-02-18, 2023-12-15] [2023-12-15, 2025-06-18, 2023-12-20, 2023-12-13, 2023-12-15, 2023-12-11, 2024-02-18, 2023-12-09, 2023-12-10, 2023-12-12] [2023-12-13, 2023-12-20, 2023-12-18, 2014-08-12, 2014-08-12, 2023-12-09, 2024-02-18, 2024-08-03] [2026-01-18 00:00:00, 2024-01-09 00:00:00, 2025-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2010-08-21 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2006-09-15 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02] [2023-12-12 00:00:00, 2023-01-15 08:32:59, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00] [2026-02-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2014-08-12 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00] [2027-01-16 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-14 00:00:00] [2026-01-18 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00] [] [2023-12-10 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2023-12-17 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00] +34 147483648 1430726838 1422055378 -150982116 2007-03-11 2023-12-15 2025-02-18 2025-06-18 \N 纯平 me for or can 赶快用于申腾 don't 一次性 0 0 28.011800 \N 0.000000 40.123000 2024-08-03 13:08:30 2023-12-19 00:00:00 2023-01-15 08:32:59.123000 2024-01-08 00:00:00 2026-01-18 00:00:00 9999-12-31 23:59:59 2024-01-08 00:00:00 2026-01-18 00:00:00 \N [0, 1, 1] [] [] [-116, -1, 5] [] [83, -55, -1] [-1, 7, 6, -52, 1] \N [5987, 8491, 245, 32767, 1, -15052, 245, 9707] [0, -22381, -28387, 3328, -25893, 32657, 32767, -1, -32312, -1] [17456, -29475, 19170, -1, -1] [32679, 1024795089, 1, 1811900604, 1, -2147483648, -1, 948495895] [2147483647, 954597305, -1] [550529538, -1, 2147483647] [] \N [-5001840667184070536, 1, 1950750441, -2583033452433339785, -1] [-590399076796638668, -8044695143040648516, 147483648, -1, 147483648, 9223372036854775807, 0, 147483648, 9223372036854775807, -1701872] [-1, -1, 32679, -1, 3789167870787786779, 0, 1, 9223372036854775807, 1742086876, 1329249648372682212] [0, -1, 1057985216, -1, 57345513, 1, -2, -2] [4, 9, -1, -2, 581537787, -638229048, 1, -1, 1, -1] [1, 7, 1] [1, -1, 0, 1, -2] [100.020000000000, 0.000000000000, 39.047400000000, 57.148600000000, 300.343000000000] [48.091500000000, 300.343000000000, 300.343000000000, 0.000000000000, 46.085800000000] [] [18.049200000000, 300.343000000000, -1.200000000000, 16.049600000000, -1.200000000000, 300.343000000000, 12.131100000000, 40.123000000000] \N [-1.2000, 20.1895, 1.2000, 300.3430, 40.1230] [0.0000, 0.0000, -1.2000] [68.1596, 40.1230, 100.0200, 1.2000, 6.0824, 40.1230, 40.1230, -1.2000] [] [0, 0, 300, 0, 18, 40, 1, 300] [75, 100, 1, 76, 0, 95, 1, 70] [] [] ["?", "然后购买专区划分郑大婚嫁吃饭了对手一周", "his she about"] [] ["-", "", "back going or going good because what", "you're she but", "do"] ["from now can't now", "look", "附近", "条件老公连接郑州环保局同力核好真正所要求", "忘不了", "-", "四方长信占有后天回服务器端人已", "元转到中庸政务静静单个有时候一级王丽出团天数前奏"] ["he's", "记下煤气标志我不会玩区域整体好友控制卡对于我", "在家"] ["-", "--", "", "?", "?", "?", "good", "we"] ["-", "", "she", "you're", "--"] ["n", "k", "p", "k", "y"] ["f", "f", "m", "w", "l"] ["m", "v", "e", "i", "c"] ["w", "f", "o"] [2025-02-17, 2025-02-18, 2023-01-15, 2023-12-09, 2023-01-15] [2027-01-16, 2023-12-15, 2024-01-09, 2024-01-31, 2023-12-20, 2023-12-19, 9999-12-31, 2023-12-17] [] [2014-08-12, 2025-02-17, 2015-01-15, 2027-01-16, 2025-02-17, 2023-12-17, 2027-01-16, 2024-02-18] [2026-02-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59] [2026-02-18 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2024-07-01 00:00:00] [] [2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2024-08-03 13:08:30] [2023-12-11 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00] [2026-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00] [2014-08-12 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00] [] [9999-12-31 23:59:59, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2024-07-01 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00] [] [2023-12-13 00:00:00, 2024-01-08 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02.123000] [2024-01-31 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] +36 1154795990 0 -2147483648 1743403554 2025-02-18 2023-12-20 2025-06-18 2024-01-31 - 中关村 -- \N the something now don't 有时候 -- 咱们轻松 \N 0 90.155100 300.343000 40.123000 55.171000 2024-08-03 13:08:30 \N 2026-02-18 00:00:00 2023-12-12 00:00:00 2025-06-18 00:00:00 2025-06-18 00:00:00 9999-12-31 00:00:00 2023-12-10 00:00:00 \N [0, 1, 1, 0, 1, 1, 0, 1] [1, 0, 0, 0, 1, 1, 1, 0] [0, 1, 1, 0, 1, 0, 0, 1, 0, 1] [-1, 39, 127, 127, 14, 0, 47, 127] [1, 1, -128] [127, 118, -128, -128, 127, 1, 1, 14] [-128, 9, 127] [-31498, 245, 0, -1, 0, 20612, -1, 32767] \N [-24404, 4091, -26780] [2177, 32767, 1299] [2147483647, 95379285, -1595644748, -1543836764, 1365614142] [-1, 32679, 638168720, -2147483648, 147483648, 1, 171406195, 2094211684] [1, -1707592172, -1474402425, 32679, -1372086696, 147483648, -428947543, -1, 1523676359, -2147483648] [] [1, 9223372036854775807, -1, 147483648, 637206423, -4373233776978804774, -1, 0] [32679, -3799738363839552666, 32679] [8377090073292913129, -9223372036854775808, 32679, 245, 147483648, -5355399460168747847, 0, 147483648] [] \N [0, 1, -2, 4, -1, 7, 6, -1] [0, 5, 0, 1, 9] [-2, 1701831803, 1, -2, 6, 0, -1, -1] [91.099900000000, 65.109600000000, 97.012400000000] [] [0.000000000000, 49.188900000000, -1.200000000000, 0.000000000000, 40.123000000000, 0.000000000000, 1.200000000000, 0.000000000000] [1.200000000000, 31.134800000000, 93.037900000000] [39.0964, 1.2000, 9.1069, -1.2000, 300.3430, 9.1240, 300.3430, 21.0821, 100.0200, 1.2000] [41.0671, 100.0200, -1.2000] [300.3430, -1.2000, 2.1069] [58.1766, 300.3430, 300.3430, -1.2000, 4.1931, -1.2000, 24.0220, 300.3430, 4.0448, 1.2000] [40, 40, 85] [32, 32, 40, 40, 40, 300, 1, 40, 51, 40] [7, 100, 300, 1, 0, -1, 1, -1, 300, 1] [-1, 0, 93, 100, 100, 80, 42, 300, 40, 8] ["正在张峰动手脚采购知道摆放一般", "k", "?", "手机号星月", "j"] ["yes", "?", "今收到", "-", "佩服"] ["则会有", "--", "v", "okay and what have", "from just he's I'm on not that"] ["k", "y", "o", "", "", "think good that's would do from think was", "that's why don't but didn't or", "昨天"] ["how now I'll when how see", "当时一点", "--", "are", "and"] ["really yeah not get want a at and say my", "抽查", "it's", "-", "can", "?", "--", "of"] [] ["well", "ok are time no why well for with", "海民"] ["o", "v", "f", "s", "u"] ["l", "a", "f", "u", "n", "d", "y", "j", "s", "j"] ["b", "p", "n"] ["v", "w", "f", "s", "r"] [2023-12-12, 2024-01-31, 9999-12-31, 2024-02-18, 2026-02-18, 2016-09-28, 2026-01-18, 2023-12-16] [2023-12-13, 2024-01-31, 2023-12-09, 2023-12-10, 2027-01-16, 2025-02-18, 2024-01-19, 2025-02-17, 2023-12-16, 2024-02-18] [2025-02-18, 2025-02-18, 2024-01-19] [] [2023-12-17 00:00:00, 2024-06-30 12:01:02, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2024-01-09 00:00:00] [2023-12-15 00:00:00, 2024-07-01 00:00:00, 2014-08-12 00:00:00] [2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00] [2023-12-12 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00] [2024-01-17 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00] [2023-12-18 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00] [2024-01-17 00:00:00, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00] [] [2023-12-17 00:00:00, 2023-12-14 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00] [2023-12-17 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2026-02-18 00:00:00] [2025-06-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00] [2025-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00] +37 -1972111069 \N -483204106 2116987795 2012-04-26 9999-12-31 2024-01-17 2025-06-18 i n ? 手册 o tell go think well don't who have be oh I'm it really going tell they see 0 1 300.343000 -1.200000 1.200000 44.082100 9999-12-31 00:00:00 2023-12-12 00:00:00 2023-12-11 00:00:00 2024-01-31 00:00:00 2026-01-18 00:00:00 2023-12-10 00:00:00 2027-01-16 00:00:00 2024-02-18 00:00:00 [0, 0, 0, 1, 0, 1, 0, 0, 1, 1] \N [0, 1, 0] [0, 0, 1, 1, 0, 0, 1, 0, 1, 1] [1, 127, -1, 55, -68, 0, -1, -128, 127, 0] [1, 1, 0, -128, -1, -128, 1, -128] [] [] [] [20036, -32747, 1, 32767, 9597] [0, -847, -32768] [1, 1, -3927] [509648793, -671799728, 32679, -2147483648, 0] [-2025184073, 2147483647, 32679, 1013733913, -2147483648, 2147483647, 2147483647, 147483648] [-1, 1, 2147483647, -1, 1659178793, 1616054461, 127810538, -2147483648] [-2147483648, 1322072673, 1, 385399235, 2147483647, 2147483647, 0, 147483648, -880379376, -2147483648] [1037845137, 32679, 7610438, -9223372036854775808, -7559655644972498072, -9223372036854775808, 32679, 108716871] [147483648, -139755110732241463, -1, 1, 245] [245, -368300, 1, 5801025528118339591, -1646751386255122333, 1, -9223372036854775808, -5836451800757139851, 3206195343056310070, 4164761737733655193] [9223372036854775807, 935439, 802379531868432413, 0, -1, 245, 147483648, -9223372036854775808, 8167420, -9223372036854775808] \N [] [1, -1, -1989378527, 0, -816055760] [-2, 5, -2, -1, 445418519] [0.000000000000, 0.000000000000, 100.020000000000] [40.123000000000, 55.172600000000, 300.343000000000] [-1.200000000000, 47.106400000000, 100.020000000000, 23.098500000000, 0.000000000000, 36.184400000000, 100.020000000000, 45.048600000000] [0.000000000000, 95.118200000000, 22.190800000000, 95.016200000000, 53.181200000000, 10.169400000000, 1.200000000000, 0.000000000000, 0.000000000000, 54.131600000000] [0.0000, 1.2000, 300.3430, 11.0470, 15.1829, 9.0539, 40.1230, 100.0200, 1.2000, 52.1888] [] [100.0200, 300.3430, 300.3430, 40.1230, -1.2000] [-1.2000, 42.1171, 100.0200, -1.2000, 0.0000, 100.0200, 36.0454, 16.0840] [76, 0, -1] \N [] [] ["t", "were", "n"] ["you're when mean", "对于我武侠陈总", "?", "well I'm were time don't okay one going", "合同气人汇众大兵处理一下"] ["反差", "留个网络考核认识成功提货解决询问过", "-"] [] ["洗液", "this", "附件查过正确及其语文叶子改掉杀毒跟单", "ok", "him", "--", "-", "?", "--", "I'm now my"] ["?", "", "know", "", "did but don't that's your got for going his get"] [] ["j", "?", "a when mean then had but", "同时一个出不了礼数最深刻去年没有到货多些青楼图形", "m"] ["b", "j", "y", "x", "t", "h", "v", "t"] ["f", "i", "m", "h", "n", "j", "e", "m", "m", "n"] [] ["r", "e", "p"] [2025-02-17, 2027-01-16, 2026-01-18, 2023-12-14, 2023-12-14] [2023-12-09, 2024-02-18, 2024-08-03, 2023-12-10, 2026-01-18, 2023-12-12, 2025-02-18, 2023-12-09] [2024-02-18, 2024-01-08, 2024-02-18, 2014-08-12, 2023-12-12, 2024-08-03, 2024-02-18, 2023-12-14, 2024-01-09, 2003-02-16] [2024-01-31, 2027-01-16, 2024-01-08, 2024-07-01, 2025-06-18, 2024-02-18, 2024-01-08, 2023-12-12, 2024-01-17, 9999-12-31] [2025-02-18 00:00:00, 9999-12-31 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59, 2014-08-12 00:00:00, 9999-12-31 23:59:59] [2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59] [2023-12-09 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2023-12-12 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00, 2023-12-09 00:00:00] [2023-12-14 00:00:00, 2025-06-18 00:00:00, 2019-06-07 00:00:00, 2023-12-14 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-16 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-17 00:00:00] [2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2024-01-17 00:00:00] [2026-01-18 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2023-01-15 08:32:59.123000, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2026-02-18 00:00:00] [2023-12-16 00:00:00, 2024-02-18 00:00:00, 2017-04-02 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00] [] [2023-12-18 00:00:00, 2018-08-21 00:00:00, 2024-01-19 00:00:00] +39 1473749940 -2147483648 -2147483648 -2147483648 2024-08-03 2024-07-01 2024-07-01 2027-01-09 - 轻信 忘记 ? 集团 所谓下图 ? 0 0 40.123000 22.170100 100.020000 20.086000 2023-12-16 00:00:00 2024-02-18 00:00:00 2023-12-10 00:00:00 2027-01-16 00:00:00 2023-12-20 00:00:00 2023-12-17 00:00:00 2025-06-18 00:00:00 2024-08-03 13:08:30 [] [0, 1, 0] [0, 1, 0] [0, 1, 0, 1, 0] [-89, -128, -72, -45, 7, 0, 1, 53] [-1, 127, 127, 0, 127] [33, 0, 2, -128, -128] [5, -128, 1, 60, 1, 93, 1, 99, 8, -1] [-22409, -24440, 1, 0, -10477] [-28135, 245, 245] [-16920, 245, -1953] [32767, -12791, 32767, -10994, 27284, 12015, -1, 3643] [] [32679, 32679, -2147483648, 2147483647, 147483648, -1, -1794593930, 300998758] [147483648, 0, 982428678, -2147483648, -1107892353, 2147483647, 147483648, -1066446539] [-2147483648, -1914192702, 32679, -1681674756, 147483648, 346190574, -1421315911, 2147483647, -2147483648, 466918478] [1, 9223372036854775807, 32679, 1, 0, -9223372036854775808, 258557234045513981, 9223372036854775807] \N [245, 3620592875310290824, 8450686101117601210, 0, -1, -1229377617, 245, -36829465, -1016452143093125617, 147483648] [147483648, 147483648, -1, 32679, 245, 1956326651692964471, 1, 147483648, -1, 0] [-1, -267023801, 515096319, 1, 813387200, -2, 0, 1] [5, -1464289665, 0] [-1, 1, 2, -1295651791, 8, 9, -1636424315, -1] [-1, -1, -1, 1, -1, 0, 6, 1, 1, 1] [0.000000000000, 1.200000000000, 87.155300000000, 40.123000000000, 3.184700000000, 40.123000000000, 40.123000000000, 1.200000000000, 300.343000000000, 300.343000000000] [0.000000000000, -1.200000000000, 100.020000000000] [0.000000000000, 300.343000000000, 100.020000000000, 100.020000000000, 67.182600000000, 69.187100000000, -1.200000000000, 83.137600000000] [6.029200000000, 36.039400000000, 56.102300000000, 300.343000000000, 40.123000000000] [1.2000, 37.0245, 1.2000] [40.1230, -1.2000, 100.0200, 300.3430, 40.1230, 55.1359, 54.1862, 1.2000, 300.3430, 47.0019] [300.3430, 78.0328, 40.0777, 88.0616, 1.2000] [] [100, 80, 0, 79, 100, -1, 40, 32] \N [18, 100, -1, 22, 0, 40, 35, 1, -1, 100] [18, -1, 78, 80, 40, 40, 0, 40] ["", "维修", "?", "?", "趋向性", "放风筝", "to", "o", "", "付清"] \N ["some", "?", "z"] ["--", "--", "", "not you're i hey his why", "注明不欢迎谈判之后心有龙成", "梁英", "果断希望海豚礼貌", "at", "did", "e"] ["just the yes could or on something your", "增值低配绿色多多雷林企业级中成", "咨询服务我们见不了青楼吃肉", "a we can a because look was get have", "也要当兵", "-", "最美丽", "-", "没有见你不可部队尾货无论令你", "h"] ["z", "right", "", "-", "-", "", "see see would well up it's he it would want", "you were see", "a can't with okay now had think are of", "真理"] ["备案", "b", "手册春子刘海洛阳落实新天智能一小时王总签不签都", "?", "信任说句去谈书生一条线少秋戴老师晓得电脑", "-", "", "we could i mean say that do up", "干干净净不然文华帮不了你聂柯所作零售", "张峰"] ["o", "s", "伴随", "?", "oh my at or had yes as"] ["p", "t", "i", "p", "c", "f", "i", "r"] \N ["a", "d", "t", "t", "j"] ["z", "r", "o", "h", "y", "y", "u", "i"] [2024-02-18, 9999-12-31, 2024-01-09, 2025-06-18, 2024-02-18, 2025-02-18, 2023-12-10, 2024-06-30] [2014-08-12, 2026-02-18, 2025-06-18, 2024-01-09, 2010-05-09, 2027-01-16, 2024-02-18, 2024-06-30, 2024-01-19, 2024-02-18] [2023-12-16, 2024-01-09, 2024-06-30, 2023-12-11, 2023-12-20, 2024-08-03, 2026-01-18, 2025-02-17] [2024-02-18, 2026-01-18, 2003-06-03, 2024-01-08, 2024-01-08] [] [2023-12-14 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2023-12-11 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02, 2027-01-16 00:00:00] [9999-12-31 00:00:00, 2024-01-17 00:00:00, 2009-01-08 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] [2024-01-08 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00] [2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-09 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2024-01-17 00:00:00] [2024-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30] [2025-02-18 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00] [2023-12-15 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00] [2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2023-12-20 00:00:00] [2027-01-16 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00] +40 \N 464220928 62691032 308694746 2024-01-31 2024-06-30 2025-02-17 2024-01-09 get 下来 中档 t ? 上网或者有空吗咩女 妈妈护 who 1 1 40.123000 66.038200 0.000000 63.011500 2014-08-12 00:00:00 2023-12-10 00:00:00 2024-06-30 12:01:02.123000 2024-01-19 00:00:00 2014-08-12 00:00:00 2023-12-12 00:00:00 2026-02-18 00:00:00 2023-12-14 00:00:00 [0, 0, 1] [1, 1, 0, 1, 0, 0, 0, 0] [] [0, 0, 0, 0, 1, 0, 1, 1, 1, 1] [] [124, 0, -12, 0, 1, -100, 87, -75, 127, -1] [] [127, 0, -128] [-32768, -25010, 32559] [32767, 1, 25045] [133, 26610, -20461] [0, -17263, -20345, 23391, -29939] [-1239993210, 818190993, 0, -2147483648, -363243294, 32679, 32679, -490027996, 259458054, 822349317] [32679, -1699382947, -1621211808] [] [-118438533, 2147483647, -342075511] [245, 0, 7487085200690479199, -4145004947222999308, -4165462310546005416] [-9083657913717342751, -1305323, 32679, -1, 245] [5294557964557609184, 32679, -4221271073682883553, 147483648, 245] [-8305208674025396191, -9223372036854775808, -1362086039, 0, 147483648, 245, 9223372036854775807, 245] [-2, 529913693, -2, 1, -1] \N [0, 133587065, 1, 0, -1, 0, 0, -658208514] [] [0.000000000000, 44.141600000000, -1.200000000000, 40.123000000000, 69.148500000000] [0.000000000000, 0.000000000000, 40.123000000000] [1.200000000000, 40.123000000000, 40.123000000000] [300.343000000000, 0.000000000000, 100.020000000000, 64.089200000000, 0.068800000000] [300.3430, 1.1716, 100.0200, 300.3430, 30.1944, 300.3430, -1.2000, 0.0000] [59.0635, 63.1084, 1.2000] [1.2000, 300.3430, 40.1230, 300.3430, 23.1294, 92.1299, 45.1495, 0.0000] [] [49, 0, 1, 40, 40, 0, 1, 32] [0, 1, 96, 300, 67, 300, -1, 0] [20, 0, 91, 94, 40] [] ["隆康常用地图直销", "?", "一会"] \N ["", "--", "you on ok me"] ["副本双电源东风路供应商曙光节省很难说心中", "o", "孙健", "--", "i", "广发处理普庆按季度", "her as going", "--", "okay come are", "考前"] ["注意不了签名火车站", "but", "?", "小欣", "this", "x", "相符", "y"] ["out", "what", "really"] ["快车", "", "go were", "d", "?"] [] [] ["r", "a", "i"] ["l", "n", "n"] ["z", "n", "w", "x", "v", "s", "t", "r", "s", "c"] [9999-12-31, 2024-02-18, 9999-12-31, 2023-01-15, 2024-02-18, 2026-02-18, 2024-01-31, 2023-12-17, 2024-08-03, 2023-12-14] [2024-06-30, 2027-01-16, 2023-12-18, 2024-08-03, 2027-01-16, 2024-02-18, 2023-12-17, 2024-02-18, 2023-01-15, 2024-01-08] [2023-12-16, 2023-12-14, 9999-12-31, 2023-12-13, 2025-02-17] [2013-07-05, 2023-12-19, 2024-06-30, 2025-06-18, 2024-01-08] [2023-12-14 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59, 2005-03-20 00:00:00, 2024-01-08 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00] [2024-08-03 13:08:30, 2023-12-12 00:00:00, 9999-12-31 23:59:59] [2023-12-17 00:00:00, 2013-08-03 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00] [] [2023-12-12 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2023-12-11 00:00:00] [2024-01-31 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2024-01-19 00:00:00, 2024-01-19 00:00:00] [] [2023-12-19 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00] [2014-08-12 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00] [2025-06-18 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-14 00:00:00] [] [9999-12-31 23:59:59, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00] +41 -2147483648 \N 1 0 2023-12-20 2027-01-16 2026-02-18 2026-01-18 just 侠诺一万国有企业返点一改宏运参观还得 收集 连接线吃饭了聪明下了南三环捷创底价 are they will mean see right what he good 次要沐浴找你琪雪检查中华挺多声卡九洲人往高处 of my all on at but at 1 1 300.343000 1.200000 1.105000 63.154500 2025-02-17 00:00:00 2023-12-11 00:00:00 2025-02-18 00:00:00 2023-12-13 00:00:00 9999-12-31 23:59:59 2027-01-16 00:00:00 2024-06-30 12:01:02.123000 2025-02-17 00:00:00 [1, 0, 0, 1, 0, 1, 1, 0] [1, 1, 0, 1, 1] [1, 1, 0, 0, 0] [] \N [2, -24, 127, 1, 127, -128, 49, -40, 7, 46] [] [-1, 127, 127] \N [245, 1, -32768, -10260, -15674, 23012, 5153, -14003] [1766, 20227, -23255, 245, -16755, 0, -1, 32767, 245, 1] [-1, -1, 245] \N [1652834015, -1, 32679, 0, -2115267924, -1541675325, 2147483647, -1265249590, 2147483647, 1] [] [-2063331048, -1674521673, -574560451, 261538613, 2147483647] [9223372036854775807, 0, -9223372036854775808, 485631039, 7702266] [9223372036854775807, 147483648, 1460618752, 245, -9223372036854775808, -794885549, -9223372036854775808, 147483648] [245, 0, -2082496027303106674, 147483648, -3713514144950729351] [-5682513003507964446, 147483648, 245, -317607910, 147483648, 245, 0, 1286069] [] [] [1680314951, -1, 0, -1, 6] [3, 0, 1, 0, -1, 0, 8, -1] \N [85.166800000000, -1.200000000000, -1.200000000000, 91.031200000000, 1.200000000000, 40.123000000000, 1.200000000000, -1.200000000000] [-1.200000000000, 5.170800000000, 0.000000000000] [] [] [300.3430, 70.0534, 300.3430, 0.0000, 98.1570] [1.2000, 40.1230, 77.0239, 1.2000, 100.0200, 1.2000, 100.0200, 1.2000, 77.1432, 300.3430] [1.2000, 0.0000, 1.2000, 0.0000, 41.0162] [26, 70, 40, 40, 40, 43, 300, 100, -1, 300] [0, 100, 22] [] [0, 44, 55, 64, 100, 80, -1, 65] ["洛阳", "天津", "--", "-", "银盘", "-", "寻找楚留香张贝属于你报账", "各位", "做梦", "礼拜正式版开封电视台发到"] \N [] ["", "处理完鸡巴夏娜不咱优势无限你好吗", "the"] ["okay going from right hey", "", "汉化", "--", "that he like", "要求", "还向", "改动服务器授狗缺货", "?", "could"] \N ["ok will can okay will for i want", "", "--", "get", "-"] ["going don't were out and yeah about", "her", "well", "表情", "i", "to", "were have really well that's really", "g", "n", "right all will me as his as"] ["l", "q", "g", "b", "q", "u", "k", "w", "m", "h"] \N ["s", "c", "o", "g", "t", "r", "d", "z", "z", "k"] [] [2027-01-09, 2027-01-09, 2024-01-31, 9999-12-31, 2026-02-18] [2023-12-11, 2026-02-18, 2024-02-18, 2024-07-01, 2023-12-10, 2023-12-15, 2017-05-22, 2024-02-18] [2025-06-18, 2014-08-12, 9999-12-31, 2027-01-16, 2023-12-13, 2023-12-15, 2014-08-12, 2023-12-09] [2024-01-19, 2023-01-15, 2025-06-18, 2023-12-14, 2025-06-18, 2014-08-12, 2023-12-16, 2023-12-18, 2024-01-19, 2026-01-18] [2023-12-12 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00] \N [] [2023-12-13 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2008-03-16 00:00:00] [2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00] [2017-11-05 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2023-12-18 00:00:00, 9999-12-31 00:00:00, 2023-12-16 00:00:00] [2023-12-14 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2025-02-17 00:00:00, 2024-01-31 00:00:00] [2023-12-12 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2024-01-31 00:00:00] [2027-01-16 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-09 00:00:00] [] [9999-12-31 23:59:59, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2007-04-02 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 2024-01-09 00:00:00] +43 -1 -1769230005 131186329 -145111229 2023-12-09 2024-02-18 2023-12-13 2023-12-16 -- so now 疙瘩 你们 you how so a 新普 1 0 \N 1.200000 -1.200000 74.152100 2023-12-20 00:00:00 2025-02-17 00:00:00 9999-12-31 23:59:59 9999-12-31 23:59:59 9999-12-31 23:59:59 2023-12-15 00:00:00 2025-06-18 00:00:00 2023-12-09 00:00:00 [1, 1, 1, 1, 1, 1, 0, 1, 1, 1] [1, 0, 1, 0, 0, 1, 0, 0] [0, 0, 0, 0, 0, 1, 0, 0] [1, 1, 1, 0, 1, 0, 1, 1, 1, 1] [-128, 55, -128] \N [-1, 0, -1, 1, 127, 1, 7, -39] [6, 3, -128, 4, 0, -111, -1, -52, -1, -44] [-18711, 0, 21241] \N [-32429, -21206, 245, 28938, -32768, 23550, -31384, -21082, 0, -5289] [9292, 8075, -16034, -11059, 0] [] [2048533015, 520475636, 0, 16690519, -27542567, -1780893631, 147483648, -1526718723] [147483648, 1194359986, -2134680583, 32679, -104244016] [-1986678466, -312743519, -1629674867] [1, -141508, -9180056386524158831, -9223372036854775808, -2844081632268889345, 0, 245, 1] [9223372036854775807, 32679, -985891470115526511, -6550609426361333003, 147483648, 147483648, -1, 0, -1, -9223372036854775808] [] [0, -3692040405460832083, 245] [8, -2, -2, -2, -1, 9, -2, -580176750] \N [0, -89027263, -2, 0, 6, -2, 1, 2] [] [-1.200000000000, 1.200000000000, 300.343000000000, 300.343000000000, 5.039900000000, -1.200000000000, 1.200000000000, 100.020000000000] [0.000000000000, 14.161900000000, 300.343000000000, 4.141400000000, 0.000000000000, -1.200000000000, 79.116400000000, 89.189500000000, 0.000000000000, 300.343000000000] [300.343000000000, 1.026500000000, 0.000000000000] [0.000000000000, 300.343000000000, 1.200000000000, 0.000000000000, 300.343000000000, 91.054000000000, 96.059300000000, 57.002500000000, 1.200000000000, 25.138000000000] [300.3430, -1.2000, 29.0927, 40.1230, 11.1649, 0.0000, -1.2000, 40.0855, 1.2000, 95.1947] [100.0200, 100.0200, 0.0000, 93.0654, 100.0200, -1.2000, 53.1269, -1.2000] [63.1478, 84.1653, 300.3430] [40.1230, 81.0652, 100.0200, 10.1330, 100.0360, 1.2000, -1.2000, 300.3430] [76, 83, 52] [100, 0, 0] [10, 0, 85, 3, -1, 21, 100, 82, 17, 100] [1, 47, -1, 100, 300, 40, 300, 56] ["产品事业部优于名家最好深夜你问都有", "--", "of", "k", "so or"] \N ["do", "been who you're from me", "come"] ["?", "r", "", "?", "?"] ["?", "-", "?"] ["前一个", "-", "博源为二路我给你打电话心思只能写上去出点老家鼎力尽请谅解", "up", "j", "go", "", "look ok how me him why so all a"] ["", "p", "-", "?", "--", "how", "", "get"] ["授权书在线预防无法名字谢谢网页太行主演", "that's", "back", "公路", "him were my like", "is so had a or this him to", "--", "--", "in him this you're are at now on", "something are how but her who okay what in"] ["u", "a", "v"] ["s", "g", "f", "u", "l"] ["b", "b", "w", "a", "l"] ["s", "f", "o", "i", "j", "r", "j", "d", "i", "m"] [2024-02-18, 2023-12-17, 2025-06-18] [2024-01-08, 2025-06-18, 2027-01-09, 2023-12-14, 2023-12-19, 2024-02-18, 9999-12-31, 2014-08-12, 2023-12-16, 2024-06-30] [] [2023-12-17, 2024-01-17, 2023-12-16] [2024-08-03 13:08:30, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00] [2026-01-18 00:00:00, 2026-01-18 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2023-12-16 00:00:00, 2024-01-17 00:00:00, 2024-06-30 12:01:02, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00] [2023-12-17 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2023-12-11 00:00:00] [2023-12-09 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2014-02-04 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00] [2024-01-09 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2002-05-10 00:00:00, 2014-08-12 00:00:00] [] [] [2023-12-14 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2008-10-16 00:00:00, 2024-01-08 00:00:00] [2023-12-09 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00] [2024-01-08 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00] +44 1822355595 -2096955821 -2147483648 -71501180 2023-01-15 2023-12-15 2026-01-18 9999-12-31 okay can't all didn't one -- 不明白天慧追求卓兴图形工作站萍聚对策千金身份证 ? 1 0 \N -1.200000 0.000000 300.343000 9999-12-31 23:59:59 2023-12-15 00:00:00 2024-08-03 13:08:30 2023-12-13 00:00:00 2023-12-14 00:00:00 2023-12-12 00:00:00 2024-02-18 00:00:00 2024-01-08 00:00:00 \N \N [] [1, 1, 0, 1, 1, 1, 0, 0, 1, 1] [-128, 1, 14] [1, 127, 0, -121, 4, 0, 0, -128, 1, 83] [1, -1, -1, 86, 7] [2, 6, -9] [-12150, -11738, 245, 22946, 245, 32767, 32767, 26898] [] [-1, -1, -32768, 245, 81, 245, 1, 245, -11745, 0] [10907, 25101, 27337, -26109, 4254, -12259, 3154, -32768] [1546047339, -2147483648, 32679, 1, 1171472342, 2147483647, -2147483648, 147483648] \N [0, 147483648, 376210749] [] \N [] [1144863672, 2594209, -8830760609694298859, -8007812155352036002, 6733028, -4007410523926317064, 2688992, -9223372036854775808] [-6253728, 142977730829427032, 1, 647945790040319478, 245] [-155173021, -461806292, -2, 1, -2, 0, 2, 1, 1, 1] [1, 0, 1134092394] [] [] \N [-1.200000000000, 35.175000000000, 1.200000000000, 0.000000000000, 100.020000000000, 50.096300000000, -1.200000000000, 300.343000000000] [300.343000000000, 0.000000000000, 100.020000000000, 100.020000000000, 300.343000000000] [] [] [40.1230, 100.0200, 40.1230] [-1.2000, 300.3430, 0.0000, 40.1230, 62.0806, 82.0999, -1.2000, 40.1230] [64.1377, 0.0000, 44.1214, 0.1912, 40.1230, 20.1531, 300.3430, 0.0000] [100, 40, -1] [100, 67, 0, 40, 40] [55, 11, 1] [] ["with no now is can of", "g", "", "did", "技术"] [] [] ["like but look", "on me do", "are", "", "打电话弄个普惠黄黎明群管理员对应从事那段清库存用眼"] ["--", "for yes", "?", "?", "-", "乔东", "天和期待保险雪松路给我关系游民", "j"] [] ["周报", "--", "国虹", "not", "with", "文字", "could", "都发生一度科贸", "", "发过"] ["say", "呵呵呵", "做出号码支持作假影响暴风影音只能对方山下", "he's", "to had up you're on want that's had was", "索尼", "尝试", "-", "好梦", "双机热备方案"] ["s", "p", "g"] ["e", "m", "j", "k", "g", "p", "n", "g"] ["s", "l", "z", "r", "h", "n", "e", "q"] ["m", "r", "v", "r", "f", "f", "d", "f", "w", "m"] [2023-12-12, 9999-12-31, 2024-06-30] [2013-03-20, 2024-01-17, 2023-12-17] [2027-01-16, 2024-01-17, 2024-01-31, 2024-01-17, 2024-01-17, 2014-08-12, 2014-08-12, 2024-02-18, 2024-02-18, 2026-01-18] [] \N [2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00] [2024-08-03 13:08:30, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2023-12-10 00:00:00, 2023-01-15 08:32:59, 2023-12-18 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00] [2023-01-15 08:32:59, 2023-12-10 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [9999-12-31 00:00:00, 2023-12-13 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-08-03 13:08:30, 2023-12-10 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59.123000] [2027-01-09 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00] [2024-01-17 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [2027-01-16 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-10 00:00:00] [2023-12-11 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 2024-01-31 00:00:00, 2023-12-17 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2023-12-09 00:00:00] [] +45 -1113690324 32679 0 -251737409 2025-02-17 2024-01-09 2023-12-16 2027-01-16 - do will want that it's here time yeah go you're good ok some mean was right r you're - 周报软件园哥们不一样屏保一级大量现货 1 0 1.200000 100.020000 63.173600 1.200000 2024-06-30 12:01:02.123000 2023-12-17 00:00:00 2014-08-12 00:00:00 2024-01-08 00:00:00 2023-12-09 00:00:00 2024-01-17 00:00:00 2027-01-16 00:00:00 2014-08-12 00:00:00 [0, 0, 0, 1, 0, 1, 0, 1, 0, 1] [1, 1, 0, 1, 0, 0, 1, 0, 0, 0] [0, 0, 1, 0, 0, 1, 0, 0] [1, 1, 1, 0, 1, 1, 0, 0] [-67, -71, 1, 127, -1] [0, 17, -1, 1, -128, -1, 1, 1, 56, 1] [4, -1, -24, 127, -128, 35, 7, -128] [-1, 1, 26] [32767, -17531, -1] [245, 1, 2257, -32768, 4790] [1, 0, -7331, 17817, 7259, 8694, 0, 9131] [28457, 17270, 1, 16831, -3508, 1, 1, 14841, 29398, 1] [1737313376, 1536327090, -1, 1168368585, -796784721, 147483648, -1, 0, 2147483647, -430407347] [-1, 1993147591, 0] [0, 2147483647, 0, 297130911, 32679, 2106722210, -608644523, 147483648, -813822954, -892754841] [32679, 147483648, -441617440, 2147483647, 32679, 2147483647, 147483648, -171101791] [1716888387, -149324, 793401627, -6928632071485694839, 32679, 245, 2420350, 0] [245, 1, 0, 32679, -1407497312653119714] [-6836330, 3356673493310895576, 9223372036854775807] [] [0, 941930663, -2, -2, 1, 0, 3, -1, -1, -1] [-2, -1426061511, 1, -1, -2] [1782582010, 0, -1404867558, -1, 0, 4, 0, -1, -1, 2] [] [100.020000000000, 100.020000000000, 100.020000000000, 100.020000000000, 37.067900000000, 100.020000000000, 300.343000000000, 100.020000000000, 300.343000000000, -1.200000000000] [40.123000000000, 40.123000000000, 300.343000000000, 300.343000000000, 1.200000000000] [34.066100000000, 59.072100000000, 53.104400000000, 85.113600000000, -1.200000000000] [100.020000000000, 65.083200000000, 0.000000000000, 65.108800000000, -1.200000000000] [-1.2000, 58.1154, 100.0200] [100.0200, 40.1230, 300.3430, -1.2000, 28.0866] [56.0814, -1.2000, 64.0466, 7.1142, 1.2000, 300.3430, -1.2000, 1.2000] [100.0200, 0.0000, 68.0329, 37.0618, 0.0000, 40.1230, -1.2000, 1.2000] [0, 5, 300, 40, -1] [] [300, 1, 70, 0, 31, 300, 1, 1] [100, 0, -1, 1, 52] ["why", "欢迎欢迎", "", "-", "见过面", "out his come but well really like", "不难明白一脚连续", "?", "will out back him yes", "视讯看法天讯几台迅驰晚上打新乡市一款"] ["you're", "-", "维修主页严重", "", "向你"] [] ["关于", "", "--", "say", "根本刘广以为你单子一条屠龙记供应商星星"] ["-", "back", "--", "were didn't to it be they for not look", "your", "处理一下", "had", "?", "look get because her tell", "在不在"] ["?", "?", "all", "p", "报价格"] ["-", "d", "k"] ["--", "a like that but they mean do I'll", "k"] ["q", "c", "x", "v", "q"] ["d", "s", "b"] [] ["b", "y", "d", "s", "d"] [2027-01-09, 2023-12-12, 2024-06-30, 2026-01-18, 2023-12-14, 2023-12-12, 2024-02-18, 2023-12-15, 2027-01-16, 2024-07-01] [2024-06-30, 9999-12-31, 2023-12-16, 2023-12-10, 2014-08-12, 2023-12-12, 2026-01-18, 2023-12-09, 2027-01-09, 2023-12-20] [] [2027-01-09, 9999-12-31, 2025-02-17, 2023-12-17, 2026-01-18, 2023-12-11, 2024-01-31, 2023-12-11] [2023-12-15 00:00:00, 2023-12-13 00:00:00, 9999-12-31 00:00:00] [2026-01-18 00:00:00, 2025-02-18 00:00:00, 2025-02-18 00:00:00] [2023-12-20 00:00:00, 2026-01-18 00:00:00, 2023-12-10 00:00:00] [2023-12-11 00:00:00, 2024-01-31 00:00:00, 2006-12-17 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02] [2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2025-02-18 00:00:00] [2023-12-09 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00] [2024-01-17 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123000, 2023-01-15 08:32:59.123000] [2024-01-31 00:00:00, 2023-12-10 00:00:00, 2023-12-10 00:00:00] [2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-09 00:00:00, 2023-12-09 00:00:00] [2023-12-18 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00] [2024-07-01 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00] [9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00] +46 147483648 147483648 -541318739 2147483647 2024-01-31 2024-01-19 2023-12-11 2023-12-18 \N 玛法 - from hey were him I'll I'll be she 我们 相恋 \N 1 1.200000 40.123000 -1.200000 1.200000 2023-12-17 00:00:00 2024-06-30 12:01:02.123000 2023-12-12 00:00:00 2024-06-30 12:01:02.123000 2023-12-19 00:00:00 2024-02-18 00:00:00 2027-01-09 00:00:00 2026-02-18 00:00:00 [] [0, 0, 0] [1, 1, 0, 1, 0, 1, 1, 0] [0, 0, 1, 1, 1] [] [1, -12, -68, -76, -128, -81, 127, -1] [127, 1, 0, 1, 127, -67, -128, -1, -8, 127] [] [1, 0, 245, 245, -17157] [245, -32768, 17946, 3616, 245, 32767, -28748, 25025, -1, -1] [32767, -32768, -1] [245, 245, 19717, -17277, -1, 14983, 32767, -4418, 245, 1] [642661284, -1961549133, -1632520261] [147483648, 1, 0, -1744138680, -1297349743, 2147483647, 32679, 1, -1859689168, 32679] [-2147483648, -682900474, 1, -927862302, 2147483647] [-349507407, 2147483647, -1, -78614223, 1, 1874067840, -1, 2091127361] [245, 0, 245, -9223372036854775808, 1, -8357268, -105910049, 2950233010911213619, 9223372036854775807, 542174562] [-5934469219472668509, -2144534872, -6914657300873320637] [] [147483648, 6639498388747507525, 1950500148, 5534235395023413274, 845004691, 1934516964, -9223372036854775808, -3846516240678010561] [-2, 1, 1, 0, 0] [-2, 1516928089, 1] [1, -1, -2] [0, -1, -1, -1709848313, 0, 1, 0, 1, 926713786, -2] [-1.200000000000, 100.020000000000, -1.200000000000, 300.343000000000, 4.012200000000, 3.072400000000, 51.118700000000, 0.000000000000, -1.200000000000, 48.163200000000] [40.123000000000, 0.000000000000, 66.093300000000, 0.000000000000, 41.133200000000, 300.343000000000, 0.000000000000, 74.031400000000] [40.123000000000, 50.037700000000, 1.200000000000] [0.000000000000, -1.200000000000, 300.343000000000, 1.200000000000, 59.112200000000, -1.200000000000, 1.200000000000, 43.061900000000, 100.020000000000, 100.020000000000] [0.0000, 300.3430, 1.2000, 27.0955, 40.1230, 100.0200, 0.0000, 40.1230] [100.0200, 40.1230, -1.2000] [] [] [1, 2, 88, 40, 0, 0, 0, 100] [100, 100, 75, 0, 36, 1, 40, 0] [] [20, 300, 1, 0, -1, 41, 40, 0] ["?", "say say was not been well then here no", "-", "--", "how"] \N ["--", "", "吃饭哪关于"] ["", "do i mean know", "", "got what do it can why want one some", "英莲"] \N [] ["d", "第一", "can one something go he's to if but", "it's something who because is you something back", ""] [] \N ["v", "l", "j", "q", "g", "k", "g", "t"] ["h", "f", "d", "r", "z"] ["z", "j", "f", "w", "v", "v", "a", "d", "b", "k"] [2027-01-09, 2011-09-25, 9999-12-31, 2024-01-31, 2023-12-10] [2024-08-03, 2023-12-12, 2024-01-08, 2023-12-13, 2023-12-09, 2001-01-02, 2023-12-19, 2023-12-09] [2024-08-03, 2023-12-11, 2023-01-15] [2024-02-18, 2024-02-18, 2026-01-18] [2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59, 2024-08-03 13:08:30, 2025-02-17 00:00:00, 2025-06-18 00:00:00] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [9999-12-31 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2023-12-16 00:00:00, 2007-01-18 00:00:00, 2024-01-19 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-09 00:00:00, 2008-01-09 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00] \N [2025-02-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00] [2023-12-14 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2023-12-10 00:00:00] [2024-07-01 00:00:00, 2024-01-17 00:00:00, 2025-02-18 00:00:00] [] +47 -1647790948 -2147483648 -781082467 1469366216 9999-12-31 2023-12-18 2024-07-01 2024-06-30 his - \N me y u ? 1 0 31.071800 83.009300 100.020000 -1.200000 2023-12-19 00:00:00 2023-12-16 00:00:00 2023-12-20 00:00:00 9999-12-31 00:00:00 2027-01-09 00:00:00 2023-12-11 00:00:00 2024-01-19 00:00:00 2023-12-19 00:00:00 [1, 0, 1, 0, 0] [1, 0, 1, 0, 0, 1, 1, 0, 1, 0] [0, 0, 1] [0, 0, 0, 0, 1] [0, 0, 9] [37, 70, 0, 97, 127] [-106, 17, 1] [86, -128, 1, -95, -53] [-23469, -1, -1, 245, 9685, 21119, -32768, 245, 1, 3919] [-9715, -1967, 28762, 6586, 2789] [27659, -31868, -27253, 245, 32767, 32767, 1, -14053, -1, 0] [0, -22701, 0, 13496, -28015, 26427, 1, 12167, 3264, 245] [892720838, -1275333461, -502213285, 32679, 1, -809431331, -1, -1] [-131778402, 2147483647, 32679, -1168492248, 1] [] [] \N [0, 147483648, 2731220997460132824, -2446609, -1, -5730779334735584973, 32679, 9223372036854775807, 0, -1053676911] [-816835933498054893, 1, -9223372036854775808] [245, -3813062339457056218, -5146461845147668118, 147483648, -1791025133, 1, 1, 9223372036854775807] [1, -2, 1, 1, -1] [1507886838, 1, 2] [] [0, 0, -1747703321, -1, -1, 1, -1, 4, 0, -140805816] [0.000000000000, 100.020000000000, 15.030000000000, -1.200000000000, 100.180000000000, 40.123000000000, 1.200000000000, 1.200000000000, -1.200000000000, 1.200000000000] [58.183400000000, 24.046900000000, 40.123000000000] [] [1.200000000000, 40.123000000000, 40.123000000000, 1.200000000000, 31.127800000000, -1.200000000000, -1.200000000000, 100.020000000000, 100.020000000000, 40.123000000000] [72.1405, 1.2000, 54.0074] [300.3430, 0.0000, 300.3430] [300.3430, 300.3430, 64.1165] [72.0955, -1.2000, 94.0153, 0.0000, 1.2000] [100, -1, -1, 26, 41, 77, 99, 27, 46, 40] [300, 30, 78, 55, -1, 1, 300, 1] [1, 40, 56, -1, 300, 75, 1, 300] [67, 40, -1, 300, 40, 76, -1, 100, 300, 0] ["think", "-", "?", "?", "小玲"] ["申请", "some", "主管切换乔总刀片服务器烟厂带来分数咨询小刘"] ["", "on", "元转到如果需问过无聊餐费", "", "your", "雪松路", "-", "just because at"] ["单价你家平志伟项目又去王睿", "and", "病毒通话一辈子场景吉林区域肛门不叫", "", "", "be", "", "忽而又去很多管理员"] ["私聊沧田探明网通销售", "do was know can't who this now see do", "达成率水哦南阳宝宝加上公章代表东创武侠朱广"] [] ["", "", "can't"] ["-", "李金才你呢岂能你们已经上班了", "--", "--", "he's see did", "?", "文件", "--"] ["c", "a", "e"] ["u", "c", "x", "d", "h", "k", "c", "v"] ["r", "c", "g"] ["t", "q", "t"] [2024-01-19, 9999-12-31, 2024-02-18, 2014-08-12, 2023-12-16, 2023-12-18, 2024-01-08, 2025-02-17] [2023-12-16, 2027-01-16, 2024-06-30] [] [2024-02-18, 2024-06-30, 2023-12-09] [] [2026-01-18 00:00:00, 2024-06-30 12:01:02, 2025-06-18 00:00:00] [] [] [2014-08-12 00:00:00, 2023-12-12 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2026-01-18 00:00:00] [2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2024-01-19 00:00:00] [] [] [2023-12-11 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2025-02-18 00:00:00] [2025-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2024-01-31 00:00:00] [2023-12-13 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00, 2024-07-01 00:00:00] [2023-12-20 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00] +48 -1760720817 -2147483648 147483648 1313932174 2025-02-18 2024-02-18 2023-12-13 2024-07-01 g 属于你 know just back me I'm why yes can't are ? - 北环不见逍遥 0 1 3.176800 -1.200000 -1.200000 0.000000 2005-05-18 00:00:00 2024-01-31 00:00:00 2023-12-18 00:00:00 2023-12-18 00:00:00 2025-02-17 00:00:00 2023-12-19 00:00:00 2026-02-18 00:00:00 2023-12-13 00:00:00 \N [] [1, 0, 1, 1, 0] [0, 1, 1, 1, 0, 0, 1, 1] [1, 7, 1] \N [-45, 3, 71, 30, 127, -128, 127, 1, -128, 127] [1, 127, -120, 41, -105] \N [1, -25634, -14570] [] [-1, -14025, 245] [-2147483648, 147483648, 147483648, 0, 1996540433] [0, -2147483648, 1] [] [-2075529733, -2147483648, 2072980064] [-1740011, -1, 5475429389733094414, -1620055036, -9223372036854775808] [-9223372036854775808, 5089927308374022035, -1, 7405862, 0] [-2022278640, 245, -9223372036854775808, 245, 147483648] [-9223372036854775808, 0, -9223372036854775808, -1, 32679, -4158792, 245, -2123393367048735685] [1, -331925954, -237543949, -2, 903456600, 1, 829605276, 2] [0, -1, -1547552776, 0, -2] [] [-1, 0, -624575134] [-1.200000000000, 300.343000000000, 1.200000000000, 4.150600000000, 1.200000000000, 97.029300000000, 61.041600000000, 1.087200000000] [40.123000000000, 300.343000000000, 10.197500000000, 73.042800000000, 24.159300000000, 76.050500000000, 300.343000000000, 40.123000000000, 1.200000000000, 19.178100000000] [40.123000000000, 1.200000000000, 85.040700000000, 40.123000000000, 100.020000000000, 1.200000000000, 1.200000000000, 16.020500000000, 100.020000000000, 100.020000000000] [-1.200000000000, 40.123000000000, 1.200000000000, 100.020000000000, -1.200000000000] [53.1173, 46.1592, 83.0377] [-1.2000, 0.0000, 300.3430, 1.2000, 40.1230, 1.2000, 1.2000, 14.1058, 1.2000, 0.0000] [300.3430, 15.1107, 40.1230, 0.0000, 300.3430] [] [11, 11, 21, 2, 37, 71, 0, 300] [1, 100, 100, 1, 100, 0, 300, 300, 67, -1] [100, -1, 0] [-1, 1, 100, -1, 13, 25, 300, 94] [] ["证书图示总监缺货平常更新没有办法刚刚年底", "收藏夹", "how"] ["产品事业部经理", "--", "", "not back do will say", ""] [] ["果断核对学习不好折扣", "and", ""] ["金色", "能早点看法现实中告诉", "?", "one", "好好", "--", "how", "大品牌"] ["we", "这三大", "it's the there", "出纳", "-"] ["of oh get to right", "k", "?", "out", "用眼"] \N ["u", "m", "m"] [] ["r", "m", "r"] [2024-01-31, 2025-06-18, 2023-12-09, 2027-01-16, 2005-01-09, 2024-01-19, 2024-06-30, 2024-01-08] [9999-12-31, 2023-12-19, 2023-12-09, 2024-01-17, 2023-12-19, 2023-12-15, 2024-01-19, 2027-01-16] [2025-06-18, 2025-06-18, 2023-12-13, 2025-06-18, 2019-11-02] [2024-06-30, 2024-01-09, 2027-01-16, 2025-06-18, 2024-02-18] [2023-12-09 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2024-06-30 12:01:02, 2024-01-09 00:00:00] [2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2025-02-17 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00] [2027-01-16 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2024-06-30 12:01:02] [2026-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00] \N \N [9999-12-31 00:00:00, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2017-12-06 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2025-06-18 00:00:00] [2027-01-09 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2027-01-16 00:00:00] [2024-02-18 00:00:00, 2016-06-16 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00] [2008-05-23 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2024-01-08 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00] +49 1941593875 1768479828 -1310805908 32679 2025-06-18 2023-12-11 2023-12-14 2024-01-19 -- 万邦从你那里 \N t 奸商 现金结就算热卖中明天你有智慧打电话 on 不低于 \N 0 0.000000 40.123000 300.343000 100.020000 2027-01-09 00:00:00 2024-01-17 00:00:00 2016-07-18 00:00:00 2027-01-09 00:00:00 2023-12-17 00:00:00 2024-01-19 00:00:00 2024-06-30 12:01:02.123000 2023-12-13 00:00:00 [1, 0, 1, 0, 0, 0, 1, 0] [1, 0, 0] [1, 1, 0, 0, 1] [0, 1, 0, 1, 1, 0, 1, 0] [88, 0, 127] [127, 10, -66, -45, -34] [1, 127, -128, -1, 127, -16, -1, 0, 1, -128] [-32, 68, 30, -91, 9] [-16731, 1, -32768, -14037, 8567, -1, 18569, 21408] [] [-1, -17548, -23865, -914, 26756, 32767, -213, 29553, 1, 5742] [-32768, -10899, 245, 8416, 5384] [1, 0, -1, 0, 32679, 295107393, 147483648, 1] [2118754852, -2147483648, 0] [-1865579784, -2147483648, -2147483648, -2147483648, -1, 147483648, 1, 1] [-1, -2147483648, 1076544603] [-903219, 0, 8176971074310666975, -1176149764, -9223372036854775808, -1, 0, 1] [-2087534944, 5674346830737447367, -4308786178050265820, 32679, 9223372036854775807, 2069715888, -7854029963835418153, -1, -6604421589274440122, 147483648] [-1, 4861454055880339036, 2817600, 147483648, -2132809514784522516] [] [1, 719196503, -1, 1, -2] [1, 1, 6] [] [1, -1, -1] [80.030300000000, -1.200000000000, 1.200000000000] [40.123000000000, -1.200000000000, 1.200000000000, 25.004500000000, 56.054700000000, 1.200000000000, 1.200000000000, 31.141400000000] [-1.200000000000, 92.141600000000, 24.063900000000] [] [300.3430, 42.0921, 49.0659] [100.0200, 1.0667, 0.0000, 75.0502, -1.2000] [100.0200, -1.2000, 55.1737, 40.1230, 0.0000, -1.2000, 100.0200, 100.0200, 40.1230, 29.1431] [300.3430, 29.0650, -1.2000] [90, 0, 20, 0, 40, 17, 79, 37] [-1, 300, 1, 40, 0, 79, 69, 27, 300, -1] [-1, 1, 100] [1, 100, -1] \N ["恶评扩展", "o", "写上去", "你不工作吗", "or", "耀博", "林慧", "ok"] [] ["我一会给大", "do she so one they do been", "新闻病毒进行到星月以下后悔还以为发生过化验", "o", "-", "?", "服装", "well he's", "生物看法", "?"] ["one", "实现罢了等着有同学心情王庄乡一年中利润", "玖佰"] ["y", "", "right yes did can't i yeah", "?", "-", "?", "", "you're when", "why want what say can't he's how in", "-"] ["一块", "--", "-", "your", "", "?", "", "all"] [] ["v", "d", "w"] ["z", "o", "g", "q", "g", "c", "h", "l", "p", "x"] ["i", "d", "e", "f", "j", "a", "w", "y", "a", "x"] ["k", "j", "v"] [] \N [2024-06-30, 2026-02-18, 2023-12-13] [2023-12-18, 2027-01-16, 2023-01-15, 2024-01-17, 2024-07-01, 2023-12-10, 2026-02-18, 2024-08-03, 2024-01-17, 2023-12-16] \N [2027-01-16 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-15 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00] [2023-12-16 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59] [2024-07-01 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00] [9999-12-31 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2024-07-01 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00] [2023-12-09 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2026-01-18 00:00:00, 2023-12-17 00:00:00] [2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00, 2023-12-16 00:00:00] [2025-06-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00] \N [2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-15 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00] +51 -1612865624 32679 -855309767 -962052581 2023-12-20 9999-12-31 2024-01-31 2026-02-18 姓江同感上不了政务饱和专区你好吗盛唐库里那些 just was what right as then have him - - l ? 中心站正泰说出来不支持看课件素材 -- \N 1 -1.200000 0.000000 300.343000 300.343000 2023-12-16 00:00:00 2023-12-09 00:00:00 2024-02-18 00:00:00 2025-02-18 00:00:00 2024-01-08 00:00:00 2025-06-18 00:00:00 2023-12-10 00:00:00 2027-01-09 00:00:00 [0, 1, 0, 1, 1, 0, 1, 1] [1, 1, 0, 1, 1] [0, 0, 1] [0, 1, 1, 1, 0, 0, 1, 1, 0, 0] [-111, -69, 1, -1, 6] [] [-24, 127, 84, 0, 1, -1, -128, 5, 2, -104] [7, 4, 127] [-4219, -32768, 245] \N [] [-1, -32768, -24609, 22586, -7719, -32768, -14851, -28937, 12877, -14851] [-1, 32679, 808146164, -2147483648, -1, -1, 971350427, 2074196913, -849205007, 147483648] [-2147483648, -1526608565, 1874775696, -1221417800, -2147483648] [2032983505, 1133107155, 406454621, 147483648, 1] [2147483647, -477967267, 990010666] [147483648, 147483648, 9223372036854775807, 32679, 9223372036854775807, -1, -586696254, -8132008] [0, 0, 245] [-1331740168, -1193140048, 245] [] [1, 1336619781, 0] [0, 3, -1, 0, 5, 182090884, -2, -1, 1, -1] [1, 1, 1, 0, 2, -1495261363, 4, -2] [0, -2, 2, -1, 0, 0, -1, -2] [80.063800000000, 0.000000000000, 0.000000000000, 64.119100000000, -1.200000000000] [46.032600000000, 0.000000000000, 18.165900000000, 100.020000000000, 100.020000000000] [0.000000000000, 96.039700000000, 40.123000000000] [1.200000000000, 1.200000000000, 39.037700000000, 40.123000000000, 300.343000000000, 80.190300000000, 40.123000000000, 100.020000000000, 300.343000000000, 0.000000000000] [60.1705, -1.2000, 300.3430] \N [100.0200, 1.2000, 300.3430] [300.3430, 0.0000, 300.3430, 1.2000, 40.1230] [0, -1, 40, 91, 1] [300, 1, 0] [] [] ["didn't", "was", "找到方向湖北那段改动天马对方写下事业", "yeah him and like for", "-", "", "-", "are like as out say I'm if well", "my his the didn't something really so think come with", "三星"] ["质保金", "合作单位每一小时完成神仙兆文铁路", "-", "喜剧", "oh oh you he's the who"] ["呀库存此菩萨也不招待费去吃饭吧张姐楚留香遮盖爱过照片等到", "?", "哥哥金汇通濮阳", "黄经理岂能切换器最美丽小姑娘赛奥", "亚太这中间快递王文非安排好王瑞诚心慢慢", "", "think", "--", "so", "like they i see"] ["作为经销商集成商授狗", "维泽", "at", "can't", "-"] [] ["访问量明星第二个兆文企业级热卖还在吗里面用十二过节", "k", "捷创三大郑旭庆白经理实现航务热卖中", "know be yes if time who", "", "创元资源宝贵长时间忽视政策东风路相见", "so", "back for can there about you at going no", "", "from"] [] ["王燕纯度", "王燕纯度通用独立群号", "", "--", "慈利达冲动客户端前提安排一周电源一般浮动我刚来呀", "?", "博威贰台影视很细那个可是中心张华", "why", "i", "just"] ["x", "e", "w"] ["o", "u", "u", "m", "p", "n", "d", "m", "e", "k"] ["z", "r", "b", "s", "h", "c", "g", "o", "v", "c"] ["b", "m", "u", "y", "i", "u", "z", "r", "z", "k"] \N [2024-02-18, 2025-02-17, 9999-12-31, 2023-12-12, 2023-12-17, 9999-12-31, 2026-02-18, 2023-12-13, 2023-12-09, 2023-12-15] [2023-12-11, 2023-12-12, 2023-12-20, 2023-12-11, 2023-12-19, 2023-12-17, 2024-01-08, 2014-08-12] [2014-08-12, 2023-12-16, 2023-01-15, 2023-12-10, 2024-01-31] \N [2025-02-17 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00] [2009-08-27 00:00:00, 2023-12-10 00:00:00, 2024-07-01 00:00:00] [2027-01-16 00:00:00, 2023-12-14 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2016-07-24 00:00:00] [2025-06-18 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00] [2023-12-13 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2006-07-05 00:00:00] [9999-12-31 23:59:59, 2023-12-19 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 2024-01-17 00:00:00] [2024-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2024-06-30 12:01:02.123000, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00] [] [2023-12-18 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-31 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2006-04-01 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00] [] +52 873021396 -2072984968 1919106694 1322925281 2023-12-13 2023-12-16 2024-06-30 2024-06-30 ? -- 亮度对策那六景泰理解北环差距很大张小莉可能手册 万普说不出日月丰我在同学家晕倒我打印出来个给你吧月底后面晚上聊 say oh right yes going then for can me did or what why right been go well why \N 1 1.200000 100.020000 0.000000 0.000000 2023-12-09 00:00:00 2025-06-18 00:00:00 2023-12-17 00:00:00 2024-08-03 13:08:30 2027-01-09 00:00:00 2024-01-09 00:00:00 2024-01-09 00:00:00 2023-12-15 00:00:00 [0, 0, 1] [] [1, 0, 1, 0, 1, 0, 0, 1, 1, 1] [1, 1, 1, 0, 1] [-38, 3, -1] [-1, 9, 0, -1, 17, 7, 78, 5, 127, 45] [-112, 127, -1, 0, -109, 1, 6, 1] [127, -112, -128] \N [0, 245, -27636, 1, 1, -180, -856, 32767] [] [] [-208835356, -1, 1899456999, 32679, 2147483647] [] [] [-2147483648, 1279386382, 1059519273, -1702202009, 32679, -568976712, 916294425, 2088950101, -1462203695, -185132726] [1660468428, -9223372036854775808, -1] \N [] [9223372036854775807, 0, 245, 245, -3082180] [1, 1, -1, -1, 0, -1, -1, -1] [-1, 0, 1, 191442609, -1196391052] [-2, -1, -1, 267416986, 0, 0, 122759508, -1066949399, 846047710, 1] [1, 8, 0, -1, -2] \N [71.029400000000, 300.343000000000, 1.200000000000] [25.005800000000, 79.176800000000, 300.343000000000, 1.200000000000, 300.343000000000, 100.020000000000, 63.088500000000, 0.000000000000] [] [] [41.1176, 40.1230, 32.1719] [61.0551, 1.2000, 0.0000] [-1.2000, 300.3430, 7.0356] [0, 40, 1] \N [] [300, 300, 1, 63, 72, 58, 1, 100, 87, 64] \N ["to", "o", "some was how it's because is"] ["--", "?", "think", "一部分以前耐心而青青绿色号发给化工厂证书", "愉快"] ["can when who something a at her of they", "徐州", "?", "", "there think really then were if up me", "no", "-", "", "what", "有机会期间有种"] ["-", "do one ok up this me yeah like on", "-", "--", "but about as yes want go when tell we say", "yeah", "she how at the me no", "--"] ["t", "?", "k", "--", "--", "when", "no want me why", "-"] [] ["it's", "关机", "便是万邦一夜看到片信任兄弟视听中旬大雪柒牌", "come oh for now when to", "十二经销商蓝色四天低配那位自私开票时机按年度"] ["y", "j", "j", "c", "t", "a", "o", "o"] ["k", "g", "j"] ["z", "d", "j", "m", "d", "u", "s", "r"] ["d", "n", "m", "j", "j", "s", "j", "e"] [2023-12-13, 2024-01-19, 2024-08-03, 9999-12-31, 2023-12-15, 2018-05-27, 2023-12-10, 2023-12-10, 9999-12-31, 2023-12-12] [2027-01-09, 2027-01-09, 2023-01-15] [9999-12-31, 2024-06-30, 2024-01-31, 2024-07-01, 2025-06-18] [2023-12-11, 2024-01-08, 2025-06-18] [] [2023-01-15 08:32:59, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2023-12-14 00:00:00] [2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59, 2003-02-25 00:00:00, 2024-02-18 00:00:00] [2023-12-11 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-12 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00] [2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00] [2027-01-09 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [2014-08-12 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00] [2023-12-15 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-13 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00] [2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00] [] +53 -2087305582 621872631 -1614211496 32679 2023-12-16 2024-02-18 2011-11-14 2023-12-20 钻石金牌成绩动画离谱哦跟你说 生病 \N - - ? - - 1 0 0.000000 1.200000 100.020000 40.123000 2026-01-18 00:00:00 2024-01-19 00:00:00 2023-12-13 00:00:00 2014-08-12 00:00:00 2023-12-09 00:00:00 2023-12-11 00:00:00 2014-08-12 00:00:00 2023-12-20 00:00:00 [0, 0, 0, 0, 0, 0, 1, 1] [0, 0, 0, 1, 0, 1, 0, 0, 1, 1] [] [] [-86, 99, 65, -36, 127] [85, 20, -128, -128, 1, 0, 0, 127] [0, -128, 1, 1, 55, -67, -32, -128, 127, 67] [-128, -128, 127, 12, 127] [0, -4574, -18347, -27346, 245, 1, -1, 25514, 0, 0] [32767, -32768, 1, -20178, 0, -1, 245, -32768, 245, -32768] [] [] [0, -2147483648, 2100805018, 1372623997, 0, -420858115, -1322439360, -1508101632, -1580661962, 1294385727] [2093807753, 147483648, 0, 1, -948292137, 147483648, 32679, 32679] [32679, 1612127388, 1992077599] [-1, 818495947, 0, 0, 2147483647, -702720558, -148320706, 1221109917] [188088, 74899605, 147483648, 147483648, 9223372036854775807, 8209842745892203412, 2254516, -3299111] \N [-2746614, 1, -9223372036854775808, 0, 32679, -1, 32679, -1513674, -1, -9223372036854775808] [-1, -1, -1, 331205613, 8986177435713568356, 2054546310, 147483648, 7831097032430832821] [641463035, 0, -1235345459, -2, -1, -1, 1, -1] [-1, -1425301282, 1, -1, 0, -2, -2, 0, 104679252, 504241816] [4, -2, -2, 1375624899, 1] [4, -1, -1, 0, -2, -1, 1207580208, 53558756] [11.146800000000, 40.123000000000, 4.074200000000] [1.200000000000, 16.105000000000, 0.000000000000] [300.343000000000, 100.020000000000, 91.186900000000, 0.000000000000, 98.118700000000, 25.179100000000, 84.006800000000, 1.200000000000, 40.123000000000, 300.343000000000] [] [1.2000, 0.0000, 100.0200] [0.0000, 100.0200, 40.1230, -1.2000, 300.3430] [] [22.1640, 300.3430, 4.0775, 93.0369, -1.2000, 0.0000, 0.0000, 0.0000, 1.2000, 40.1230] \N [40, -1, 1, 40, 38] [-1, -1, -1, 40, -1, -1, 90, -1, 0, 0] [71, 40, 300, -1, 1, 40, 72, 40] ["?", "out were can't", "--", "认识压力行踪出不了提醒炒货机型算是大众", "--", "out", "o", "", "衣服", "in look look a like i as"] ["?", "j", ""] ["but", "六万机会经常留不住儿孙们做私单档次", "", "i", "but", "get if yeah", "h", "客户"] ["不让", "女孩", "--", "good did the here say his the that", "not"] ["还有呢", "me", "yeah", "why back I'll hey there at all but", "拜托其他我们脱机吉林省一份", "?", "陈老师", "would", "不好", "I'll"] [] ["yes just the got yes okay we it's now", "这话", "-", "我一会给你打", "just", "x", "one", "购买专区最大化晓得作假下个礼拜有些"] ["", "that's right that's can got mean", "制胜也冲击梦幻水族博客中小企业说句如此看到片", "from you go go", "把握"] ["j", "h", "d", "d", "b", "z", "k", "f"] ["q", "n", "c", "e", "c", "b", "u", "i", "u", "c"] ["f", "q", "j", "f", "t"] ["o", "w", "y", "z", "z", "j", "k", "w", "v", "x"] [2024-01-19, 2023-12-17, 2027-01-09] [2027-01-09, 2024-08-03, 2023-12-13, 2024-08-03, 2026-02-18] [2024-08-03, 2023-12-11, 2023-12-16] [] [2024-02-18 00:00:00, 2024-06-30 12:01:02, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00] [2023-12-19 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [2023-12-18 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00] [] [] \N [2027-01-16 00:00:00, 2018-07-25 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2008-11-03 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00] [2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00] [9999-12-31 23:59:59, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 2026-02-18 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00] +54 -1080076805 -1791017957 0 2147483647 2024-01-08 2023-12-13 2027-01-09 2024-06-30 置业 开票春江花月夜多少最好关闭 宝莱开始对于硕博线缆失恋有些王平 all would yes will me will why so go there 马马虎虎等着李景林翻译说声讲的就行通用分忧 b 发布燕龙在哪里呢机柜智博经营回访分销商 yeah here hey be then don't 1 1 1.200000 0.000000 0.000000 66.120100 2023-12-18 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2023-12-15 00:00:00 2024-01-17 00:00:00 2023-01-15 08:32:59.123123 2006-09-21 00:00:00 9999-12-31 23:59:59 [0, 1, 0] [0, 1, 0, 1, 1] [1, 1, 1, 0, 1] [0, 0, 0] [4, -1, 127] [3, -128, -1, -31, -116] [-128, -2, 6] [7, -128, 121] [8521, -22836, 12238, -32768, 26727, 29004, 245, -30781] [] [-31185, -31958, -16995] [] [-2147483648, -1185259373, 1192514860, 0, 1] [-2147483648, 2147483647, -1, -2147483648, -782332358, 255548816, 485958295, -882453780, -1082425870, 0] [1761553589, -1069147976, 1, -1646050605, 2147483647] [-1831845574, -777730870, -1122648528, 0, 147483648, 1779487269, -500890822, 1] [32679, 1351659177351501386, -7480390, -1, 147483648, -1937942131, 0, 147483648, 3909611433543527114, -8003538] [-1264718301, 9223372036854775807, 3555412565833919344] [] [2458651604121026643, -1079868425120616792, -2521432] [-377998566, 0, 596120973, 0, -1, 1, 0, 0, -60382392, 3] [0, 7, 0, -1, 0, 0, -2, 0] [0, -1, 1531012002, -2, 648096124, 1, 574658912, -1, -1, 1] [0, 1, 1016612663, 1429229368, 7, 1, -1, -1, 0, 0] [1.200000000000, 0.000000000000, 40.123000000000, 100.020000000000, 20.137700000000, 40.123000000000, 300.343000000000, 21.048800000000, 49.194800000000, 40.123000000000] [-1.200000000000, 0.000000000000, 1.200000000000, 40.123000000000, -1.200000000000] [100.020000000000, 0.000000000000, 100.020000000000, 68.005600000000, 98.184000000000, 300.343000000000, 100.020000000000, 86.077500000000] [89.112500000000, 87.068400000000, 0.000000000000] [-1.2000, 60.1361, 86.0883] \N [] [61.0635, 63.0031, 1.2000] [300, 90, 300, -1, 300, 1, 0, 1] [54, 40, 23, 0, 300] [] [1, 1, 1, 99, 1] ["was all been can't can't it could", "每个人很好用鉴别", ""] ["汇祥会想南三环一张心情伴我到回家", "of", "v", "", "", "-", "", "okay", "不小随风耀博回复招工金总天浩普庆经销商深爱", "总业绩"] ["", "v", "河南区硅谷博客", "her tell how", "也好", "很好用", "?", "--"] ["like", "will", "a", "清理", "分钟", "", "", "--", "--", "划分"] ["didn't like back there", "as it will have if come yeah know for", "-", "-", "is then get is of could can't say be can"] ["been don't ok i", "what there go no her up have one", "say", "隆康谁知道客服天数确定人往高处扩充", "张海燕员工你说颜色东风系列发过手册单条女人惭愧", "time something can the", "a", "?"] ["--", "", "you from"] ["-", "-", "?"] \N ["u", "b", "p"] [] ["q", "x", "d", "y", "j", "t", "e", "z", "s", "o"] [2024-01-19, 2023-12-09, 2023-12-09, 2023-12-09, 2025-06-18, 2024-02-18, 2014-08-12, 2026-02-18] [2025-06-18, 2026-01-18, 2023-12-09, 2024-01-31, 2023-12-15] [2024-01-17, 2025-06-18, 2024-01-17] [] [2024-07-01 00:00:00, 2027-01-16 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2024-02-18 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00, 2024-01-08 00:00:00, 2023-12-13 00:00:00, 2014-08-12 00:00:00] [2024-01-09 00:00:00, 9999-12-31 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00] [] [2023-12-20 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2024-01-19 00:00:00, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2007-05-26 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00] [2024-01-31 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-13 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2024-01-09 00:00:00] [2023-12-13 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2023-12-16 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [] +55 0 -1699610688 50724013 1167629106 2023-12-11 2025-02-18 2023-12-11 2023-01-15 -- 得劲创世纪青春串货置业戴老师逝去回事机会你是 配置 -- were my it's about ? ? 0 1 \N 88.088500 33.153300 27.126100 2026-02-18 00:00:00 2026-02-18 00:00:00 2023-12-19 00:00:00 2024-08-03 13:08:30 2025-02-17 00:00:00 2023-01-15 08:32:59.123123 2027-01-16 00:00:00 2024-01-19 00:00:00 [] [1, 0, 0] [1, 1, 0, 1, 1, 1, 1, 1, 1, 0] [1, 1, 0] [-128, -1, -128, -1, 4] [-1, 0, -1] [] [] [740, -32768, 0, -32768, 1, -32768, 15980, 1] [-32768, 9421, 245] [-32768, 24996, -32768] [] [-1, 0, -2147483648, 401043029, -1] [776852047, -708075837, 32679, -1389203475, -1315575976] [32679, 673593105, 1931974663, -1746064647, -471258287, 32679, -1, -1, 519660396, 2041795412] [1619107812, 0, -2147483648, -1042902805, 287758494] [1, -985948596, 9223372036854775807, 9223372036854775807, 4095653849088998740, 4985800289449093271, 9223372036854775807, -1, 8318223158976010419, -2606872223537107843] [4357369, 1609503209, 9223372036854775807, -9223372036854775808, 1, 7081736351443288769, 7981206, 0, 9223372036854775807, 848816916629542887] [1576928867, -1, 32679, 1, 2129691017, 32679, -6501687727398760100, 6710767] [] [0, 0, 0, 242943943, 1, 1, -2, -1112098323, -1, -1] [-29665554, 8, -2, -1736139310, -2] [-1089549817, -2, 0, 3, 0, 1, 1, -2] [1, 1, 1] [0.000000000000, 53.031600000000, 1.200000000000, 16.063400000000, 0.000000000000] [76.145000000000, 64.120900000000, 0.082100000000, 1.200000000000, 50.043700000000, 1.062000000000, -1.200000000000, 27.105900000000, 0.000000000000, 40.123000000000] [40.123000000000, 100.020000000000, 26.009200000000, 40.123000000000, 300.343000000000] [300.343000000000, 85.164400000000, -1.200000000000] [300.3430, 1.2000, -1.2000, 71.1618, 1.2000, 43.1853, 40.1230, 0.0000] [100.0200, 87.0172, 300.3430] [100.0200, 300.3430, 23.0049, 0.0000, 40.1230] [] [300, -1, 0] [40, 15, 0, 72, 33, 43, 0, 0, -1, 40] [-1, 27, 40] [40, 100, 35] ["right", "that", "about"] ["?", "可选", "--", "人数", "有限", "?", "旭日河南总经销捷创隆康自己的我就网吧一句话知己热点", "who", "-", "通道"] [] ["", "I'll i him that's back time", "故事", "粘贴礼拜多点游戏套件熟人严格", "呵呵呵你杀时间回来烂漫这个十分河南区保存妈的怀念月底", "志强", "can come I'm go they", "why", "would", "come"] ["", "历来", "know", "", "o"] ["your all your but", "are", "-", "李静开炮一辈子自动化热卖企能三个你的打我", "good say because then out get"] ["", "交到你不蓝奥库存周报引进过来对话框区号", "--"] [] [] ["b", "t", "w", "y", "g", "z", "h", "v", "f", "h"] [] ["o", "b", "v", "h", "r"] [2023-12-09, 2026-02-18, 2024-06-30, 2025-02-17, 2023-12-15] [2023-12-17, 2023-12-13, 2024-01-09, 2025-06-18, 2027-01-16, 9999-12-31, 2023-12-17, 2024-01-17, 2014-08-12, 2026-01-18] [] [2027-01-09, 2024-06-30, 2025-06-18, 2017-07-17, 2024-01-08, 2025-02-18, 2014-08-12, 2024-06-30] [] [2023-12-17 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59] [2024-01-17 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2023-12-11 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00] [2024-01-19 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00] [2023-12-14 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2023-12-14 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 9999-12-31 23:59:59] [2023-12-18 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2008-11-10 00:00:00, 2009-12-06 00:00:00, 2023-01-15 08:32:59.123123] [2023-12-09 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00] [9999-12-31 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 2023-12-14 00:00:00] [] +56 -1 1099221057 1655716221 1173182936 2026-02-18 9999-12-31 2023-12-11 2023-12-16 from ? -- were out 大中华滋味相处自导泰安肥肉谁做早点中小东大街 - 热销按年度病毒更多 \N 0 1.200000 40.123000 93.180100 300.343000 2025-06-18 00:00:00 2024-01-31 00:00:00 2023-12-19 00:00:00 2025-02-17 00:00:00 2025-02-18 00:00:00 2023-12-09 00:00:00 2027-01-09 00:00:00 2025-06-18 00:00:00 [1, 0, 0, 0, 1, 0, 0, 0] [1, 1, 0, 1, 0, 0, 0, 1, 1, 1] [0, 1, 0, 1, 1] [] [] [-1, 1, 82] [-27, 2, -1, 0, -18] [-1, 122, 6] [0, 245, 245, 10371, 32767, -11324, 14674, -13845, 17047, -1] [-32768, -499, 0, -32768, 32767] [5462, 19973, 32767, 1, 5701] [] [20916382, -1525386105, 442994721] [147483648, 1761079681, -2147483648, -515260792, -1, 0, 32679, 2147483647, 1766160851, 1] [313777957, -2147483648, 2147483647] [1798102266, 1468051165, 1] \N [147483648, 9223372036854775807, 147483648, -6525740724722998802, 1] [] [-9223372036854775808, 1730196839, 147483648, 0, -5406805331417948818, -1601873, -859693625, -134552275643911512] [0, 0, -1] [-1, 4, 0, 0, 0, 0, 4, -1] [-1, 1000118538, 1, 0, 5] [] [72.133900000000, 100.020000000000, 100.020000000000, 1.200000000000, 74.147800000000] [26.189700000000, 40.123000000000, 80.160700000000] [67.134300000000, 100.020000000000, 1.200000000000, 100.020000000000, 19.030700000000, 300.343000000000, 40.123000000000, 29.043300000000] [] [84.1146, 1.2000, 0.0000, 84.1933, 100.0200, 100.0200, 54.1111, 100.0200] [58.1182, 52.0074, -1.2000] [53.0474, 1.2000, 27.1372, 0.0000, 0.0000, 40.1230, 300.3430, 0.0000, 40.1230, 40.1230] [0.0000, 1.2000, 0.0000, 70.0073, 56.0941, 300.3430, 13.1218, 10.1626] [-1, 87, 300, 1, 57, 0, 40, 4] [22, 91, -1, 0, 0] [] [] ["were", "i", "that", "are", "叁仟", "yes think think didn't like oh his really", "-", "周岁重命名扩展王海焦作市", "海涛", "from here who going think out as"] ["最上面主管赵洪应宝宝热卖其余", "最近忙什么呢", "okay well"] ["--", "q", "仔细招待惠达", "提成点中原路", "海创青青绿色开车原厂大企业我不", "怀念也要铭岳汇宝交货毕业墙核算屏蔽嘉运", "here that so is tell been", "在呀加入结果封库咱用这种留一下新买在那在欧普", "?", "--"] [] ["from", "多久老公感悟周伟燕很难说", ""] ["还没见不会被人家走走", "?", "", "吃饭了", "--", "--", "现货你有智慧打电话瑞丽绿色先打款所有哪方面可是彩页", "泡茶万能淀粉汝南人民红色"] [] ["to", "失误就不所有安信平均天数郑州市不走百度你也来了", "就在图站去不了冤枉顺驰自豪不想", "--", "?"] ["p", "l", "a", "u", "d", "d", "t", "l"] ["y", "i", "o", "r", "t"] ["y", "v", "o"] ["n", "t", "l"] [2024-02-18, 2024-01-08, 2023-12-12, 2023-12-18, 2023-12-11, 2024-01-09, 2023-12-17, 2026-01-18] \N [] [2023-01-15, 2025-06-18, 2023-12-16, 2023-12-09, 2024-02-18, 2005-04-22, 2023-12-18, 2027-01-09] [2024-01-19 00:00:00, 2023-12-13 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00] [2025-02-17 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02, 2023-12-20 00:00:00, 2024-01-31 00:00:00] [2023-12-18 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00] [2025-06-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2025-06-18 00:00:00] [2026-01-18 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2026-02-18 00:00:00] [2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00] [2026-01-18 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00] [2024-07-01 00:00:00, 2023-12-14 00:00:00, 2023-12-20 00:00:00] [2023-12-19 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-16 00:00:00, 2023-12-17 00:00:00] [9999-12-31 23:59:59, 2026-01-18 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2002-03-17 00:00:00, 2017-04-23 00:00:00, 2023-12-10 00:00:00] [] +57 170079656 147483648 -2086239824 1537893898 2024-02-18 2027-01-09 9999-12-31 2026-02-18 go or think good go get up in with she -- ? 里面 大概 ? - how 1 0 -1.200000 72.120500 1.200000 -1.200000 2024-02-18 00:00:00 2026-02-18 00:00:00 2023-12-14 00:00:00 2024-08-03 13:08:30 2023-12-15 00:00:00 2024-07-01 00:00:00 2026-02-18 00:00:00 2024-01-31 00:00:00 [0, 0, 0] [1, 1, 1, 1, 0, 1, 1, 0] [1, 0, 0, 1, 1, 0, 1, 1] [1, 1, 1] [2, -128, -1, 127, -14, 0, 9, -128, 3, -128] [0, 1, 127, 37, 0, -1, 26, 1, -128, 3] [-1, -128, -42] [127, -108, 7, 1, -128, 1, -66, -128, 52, -74] [-1, 15323, 16082, 1025, -32768, -32768, 245, -24508, 6606, 245] [-6412, 245, -27981, -1, 0] [] [-32768, 1, 14496, 1153, 18022] \N [2016035273, -1, 147483648, -1833041902, 2147483647, -2147483648, -1103189736, 2147483647, -1338652078, 147483648] [] [83700970, 1700427816, 1] [245, 4638496495092319877, 1706595384, -1, -9223372036854775808] [0, 245, 245] [-1, 2870788, 356085989, -408776656751724869, -9223372036854775808, 3511823390479749033, -1347741, 245] [] [-1320421572, -684160549, -2, 0, -2, -2, 0, 0, -2, 1522713434] [-2, -1, 1, -2, -510280277, 6, -1, 0, -1639534241, 0] [-2, -1, -1, 1, 1, 0, 3, -1, -1961451179, -2] [4, -1, -2, 1, 3, 0, 3, -2, -2, -2] [-1.200000000000, -1.200000000000, 0.000000000000] [300.343000000000, 96.003200000000, 1.200000000000] [-1.200000000000, 91.163600000000, 1.200000000000, 40.123000000000, 85.023300000000, 79.006200000000, 1.200000000000, 40.123000000000] [] [100.0200, 40.1230, 78.1996] [97.0793, 0.0000, 50.0053, 50.0585, 0.0000, 40.1230, 24.0230, 0.0000] [96.0763, 76.1982, 74.1455, 100.0200, 40.1230, 31.0634, 40.1581, 79.0879, -1.2000, -1.2000] [100.0200, 7.1305, 1.2000, 300.3430, 58.1783, 83.0162, 1.2000, 29.1643, 300.3430, 12.1696] [9, 1, 1] [8, 58, 13, -1, 300] [1, 11, 66, 300, -1, 40, 40, -1, 81, 80] [100, -1, 49, 40, 6] ["-", "can't me well", "?", "--", "比如说本来反应不谈输得起诊断跟单", "--", "桌面", "跑跑卡丁车"] ["地矿研究所", "with", "表达招商尽量五级领先擅作主张正品", "--", "-", "going just but it's on been it's think", "him", "一共"] ["think", "医院人体看吧麻烦", "-"] ["-", "方法区域年后未来部门九州有谁", "up", "惊风深蓝不能说数字显示意义公里关闭五级研究院文件夹", "k", "-", "游民", "伤和气热插拔木马暂时信源不完高价", "?", ""] ["", "?", "", "--", "备份文件", "don't", "--", "散落的张贝改动刻录过年欧阳行政责任"] [] [] ["", "him can't know can't just then", "no"] \N ["q", "d", "z", "q", "q", "r", "t", "g"] [] ["q", "v", "q", "i", "m"] [2024-08-03, 2024-02-18, 2024-01-19, 9999-12-31, 2023-12-16] [2025-06-18, 9999-12-31, 2024-07-01, 2024-01-09, 2024-07-01, 2026-01-18, 2027-01-16, 2023-12-16] [2023-12-09, 2018-08-19, 2024-07-01, 2025-02-17, 2023-12-19, 2014-08-12, 2024-01-17, 2024-01-09] [] [2014-08-12 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00] [2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00] [2024-07-01 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59] [2024-06-30 12:01:02, 2023-12-16 00:00:00, 2025-02-17 00:00:00] [2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123000, 2024-07-01 00:00:00, 2023-12-13 00:00:00, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2023-12-12 00:00:00] [2027-01-16 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00, 2024-01-19 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00] [2024-01-17 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00] [2023-12-12 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00] [2025-02-17 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 9999-12-31 23:59:59] [2014-08-12 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00] [2023-12-15 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00] [2011-02-22 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59.123123] +58 -1 2147483647 -367796061 336196150 2024-01-19 2025-02-18 2024-02-18 2024-06-30 -- 一万一张过去怎么样文件寻找那天 l 还得 if so I'm can't 规定 疑问稍后累了委托收款随时统记表北站小雨 -- 1 1 \N 40.123000 1.200000 0.000000 2023-01-15 08:32:59.123000 9999-12-31 00:00:00 2024-01-17 00:00:00 2023-12-12 00:00:00 \N 2024-01-19 00:00:00 2014-12-16 00:00:00 2023-12-17 00:00:00 [0, 1, 1] [] [0, 1, 0, 1, 0] [] [1, 0, 1, -1, 20, 127, -52, -104, 0, 95] [0, 2, -80, -128, 5, -128, 24, -1, 127, -1] [2, -128, -1, 127, -33, 0, 96, 97, 4, 1] [10, 127, 75, 1, 0, -98, -128, -85] [0, 245, 32661, -9381, -21458, 25855, 245, -7773, 31724, -23868] [0, 6654, 27379] [] [245, -19899, 18252, 1, -9383, -31171, 16925, -3711] [] [1, 373600103, 1312376449] [0, 1973160898, -1389837711, 2147483647, 147483648, -1691286497, -365694833, 719116358] [] [-9223372036854775808, 147483648, 0] \N [147483648, 245, -9223372036854775808, -4584620616392667242, -8030387, 7386824980301848455, 1, -9223372036854775808] [-1385654110, 1, -1, 7659158, 5647566415873640494, -1551740842, 5575275538262239881, -7003699, 245, 245] [1, -2, 1, -2, 1186446474] \N [9, 1, -1, 6, -2] [] [99.092100000000, 0.000000000000, 12.038100000000, 0.000000000000, 1.200000000000, 40.123000000000, 21.096900000000, -1.200000000000, 56.002800000000, 12.120800000000] [40.123000000000, 40.035600000000, 1.200000000000] [] [] [75.1667, 40.1230, 100.0200, 0.0000, 0.0000, 300.3430, 1.2000, 11.0110] [100.0200, 300.3430, 0.0000, 93.1690, 1.2000] [40.1230, 85.0175, 8.0180, 82.0411, 1.2000, 73.0785, 40.1230, 35.1863, 40.1230, -1.2000] [100.0200, 1.2000, 40.1230] [92, 1, -1, -1, 40] [] [1, 300, 3, 100, 0, 40, 0, 49, 300, 67] [-1, 300, 1, 100, 0, 100, 77, -1, 7, 40] ["why", "焕然", "-"] \N ["mean", "看课件素材", "利盟攻防总经销商手机号核心主管网上讨论组你的号码", "in I'll how but going", "-"] [] ["-", "高杰感觉", "哭泣", "-", "when", "-", "仁信", "what get mean his", "所谓", "你说杀了一切运输付出过程序马马虎虎原因王海每一小时太原"] ["?", "?", ""] ["I'm just her so how she up ok could", "don't", "know no is will that were didn't time", "电子每次朱广询价", "really", "because so out see", "--", "don't"] ["的风格", "you're want just see but or to not of", "", "-", "her good", "?", "", "--", "文杰", "信用社"] ["b", "l", "x", "v", "e", "m", "k", "z", "e", "c"] ["l", "g", "n", "f", "e", "f", "k", "d"] ["q", "i", "o"] ["j", "d", "l", "f", "q", "r", "v", "u"] [2024-01-09, 2024-08-03, 2023-12-15, 2024-01-09, 2023-12-19, 2023-01-15, 2023-12-19, 2023-12-10] [2023-01-15, 2024-02-18, 9999-12-31] [] [2023-12-17, 2004-01-23, 2012-07-21, 2024-01-19, 2024-02-18, 2023-12-14, 2025-06-18, 9999-12-31] [2014-08-12 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00] \N [] [2024-01-09 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00] \N [2025-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00] [2023-12-12 00:00:00, 2023-12-11 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00] [2014-08-12 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59.123000] \N [2023-12-09 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2025-02-17 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2001-07-06 00:00:00, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2024-07-01 00:00:00] [] [9999-12-31 00:00:00, 2007-01-22 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-18 00:00:00] +60 1 1 1400009371 -1543545878 2014-08-12 2024-01-17 2023-12-15 2023-12-09 - 初次五级手机号 see go was 南村 l 玩游戏先达含有发放批发价收吧对话框针对兆文自私 ? 0 0 \N 1.200000 40.123000 21.020300 2023-12-14 00:00:00 2023-12-11 00:00:00 2023-12-12 00:00:00 2024-01-08 00:00:00 2024-02-18 00:00:00 2024-01-08 00:00:00 2024-01-17 00:00:00 2023-12-18 00:00:00 [0, 0, 1, 1, 0, 0, 0, 1, 0, 0] [1, 0, 1] [0, 0, 0, 0, 0] [0, 0, 1] [-1, 79, 127, -95, 1] [-100, 97, 1, 121, -128] [1, 0, 87, -49, -1, 0, 38, 127, 1, 28] [60, -91, -12] \N [-18620, 9263, 1, -23893, 245] [1, 0, 1, 2520, 8556, 1997, -32768, 9766] [] [-1, -1, 347558199, 1074528139, 1473405480, 1, -1263592215, -1771289325] [-1684082260, -1927880292, 1551163664, -1402558155, -2147483648, -131100343, 858972410, 147483648, 0, 0] [32679, -1664567647, -536350121, -2147483648, 147483648, 1346945181, 0, -2147483648, 0, 1464442969] [-2147483648, 0, 621008559, 0, 0, 32679, -1848831258, 2147483647, 0, 0] [0, 7773796554014609478, 8504270169319015068, 32679, 0, 418327720, 1, 0] [] [5903290217400355135, 2144672457, 1757419916950778571, 4920652502439501061, -1] [-9223372036854775808, 9223372036854775807, 32679, 9223372036854775807, 9223372036854775807, -1, 8180261371720498557, 32679] [0, 1, -2] [1, -1, -1, 1, -2] [1, -1660074862, 1, 1, 1669262749, 727961473, 0, 1509343458] [6, -1387432556, -2] [40.123000000000, 92.124500000000, -1.200000000000, 75.162200000000, -1.200000000000, 300.343000000000, 0.000000000000, 0.000000000000] [76.123300000000, 93.128200000000, 72.004700000000, 100.020000000000, 49.085100000000] [40.123000000000, 0.000000000000, 0.000000000000] [-1.200000000000, 40.123000000000, 0.057000000000, 300.343000000000, 66.174000000000] \N [100.0200, 300.3430, 40.1230, 300.3430, 100.0200] [0.0000, 49.1226, 39.1413, 40.1230, 58.0866] [99.1808, 0.0000, 1.2000, 62.1548, 0.0000, 0.0000, 31.0726, 100.0200, 300.3430, 27.0688] [1, 24, 0, 1, 40, 100, 40, 40] \N [-1, 300, -1, 1, 79, -1, 40, 30, 300, 80] [1, 90, 0] ["didn't", "yes that's one back she what", "-", "-", "me say get tell something for think"] [] ["going", "o", "will", "going", "", "me as I'm no go get the time well like", "up", "d"] ["out I'll don't from", "i", "but when can't if all you I'll tell his I'm", "纵观", "这种腾讯正厅周经理全系列又给歘俩勇气", "全国天宇平志伟景峰询问精英们惭愧", "when from", "港湾", "g", "--"] ["w", "-", "-", "-", "?"] ["had", "本公司当地砖石表格也是俩年好的", "-"] ["there will they really is", "I'll", "?", "how can", "?", "", "--", "be"] ["-", "?", "词句"] ["s", "p", "x"] ["e", "d", "u", "c", "n", "h", "c", "f", "k", "t"] ["f", "b", "x", "l", "n", "z", "b", "n", "v", "c"] ["e", "n", "t", "u", "j", "a", "o", "m", "p", "o"] [2023-12-18, 2024-07-01, 2024-01-17, 2026-01-18, 2025-02-18, 2026-02-18, 2024-08-03, 2026-02-18] [2025-06-18, 2023-12-19, 2024-01-08] [] [] [2025-06-18 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2023-12-14 00:00:00] [] [9999-12-31 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2026-01-18 00:00:00] [2024-01-19 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2024-08-03 13:08:30] [2023-12-20 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [2023-12-18 00:00:00, 2023-12-13 00:00:00, 2023-12-10 00:00:00] [2023-12-13 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2026-02-18 00:00:00] [] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-16 00:00:00, 2024-08-03 13:08:30, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-10 00:00:00] [2023-12-17 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 2007-11-12 00:00:00, 9999-12-31 23:59:59, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00] [2025-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00] +62 193042677 0 0 410327996 2023-12-17 2024-01-09 2014-08-12 2025-02-17 高兴 疑问 can't go are have be were think mean and do 结果 伟业睡觉设和有时间 you no good here was look not -- 0 1 -1.200000 1.200000 -1.200000 40.123000 2024-01-19 00:00:00 2024-01-19 00:00:00 2024-08-03 13:08:30 2023-12-12 00:00:00 2024-02-18 00:00:00 2023-12-15 00:00:00 2024-06-30 12:01:02.123000 2023-12-12 00:00:00 [1, 1, 1, 0, 0, 1, 0, 0, 1, 0] \N [] [] [] [-70, 0, 127, 33, -128, -128, 0, -6] [-78, 101, -1, -1, 45, 5, 30, -33] [4, 1, -116, -43, 7, 3, -1, -128, 5, 0] \N [-1862, -1, 2535, -1, -20103, -25450, 28195, 19222, 245, -954] [26151, 1, 32767, -1, 23592, 20777, -993, -5811, -16799, 0] [] [32679, 512296354, 2147483647, 32679, -1926357095] [-1, 2147483647, -1472397868, 237975025, 147483648] [2147483647, 147483648, 32679, 521781460, 578155365, 465015618, 809067185, 722041796] [-209485618, 0, 32679, -1943445930, 0] \N \N [1, 147483648, 0] [3339191341891312611, 32679, -1, -1, -1, 245, 32679, -9223372036854775808, 1, 0] [-1, 2, 1] [0, -1975075861, -1] [-2, 0, -546910392, 77421439, 3] [1, -2, 0, -2, 1756363272, 6, -1, 0] [] [0.000000000000, 2.195600000000, 34.138600000000] [300.343000000000, 52.194000000000, 1.200000000000] [0.000000000000, 97.143200000000, 84.092300000000, 300.343000000000, -1.200000000000, 49.081500000000, 16.071400000000, 97.166400000000] [6.1150, 0.0000, 40.1230, 40.1230, 7.0882, 52.1866, 14.1439, 1.2000, 300.3430, 300.3430] [0.0000, 300.3430, 40.1230, 1.2000, 1.2000] [15.1072, 1.2000, 40.1960] [100.0200, 50.0693, 1.2000, 73.0129, 100.0200] [] [0, 1, 100] [300, 300, 28, 40, 300] [100, 5, 16, 100, 53] \N ["--", "?", "晒后版本不爱", "when something up I'm was", "", "配置路由器照顾拜访商厦任性", "have would get her like think", "come", "院内", "me tell"] ["good", "", "go", "how with him no had", "-", "均为", "第三方看不出来河南分区员工小丽区号间进来编号高密度", "-"] ["on about for", "不欢迎收藏夹学习不然惊喜金牌首先就在回忆奇偶", "at", "g", "now she right been come up back get no you're", "最低", "s", "there want"] ["稍后", "-", "有车吗", "g", "-", "--", "like", "赛奥请重试词库梅捷关机没沈晓海"] ["it's", "", "很错", "出纳早上群里海民根本婚嫁张姐无关", "--"] ["李静", "男人", "-", "on", "成功", "your", "not will there tell going of with look", "?"] ["did will that did the his me", "加上批发价唯一的王总王瑞品放", "going"] [] ["f", "d", "c", "q", "j", "u", "v", "u", "k", "f"] [] ["n", "o", "z", "q", "f", "y", "u", "l"] [] [2023-12-16, 2023-01-15, 2024-07-01, 2023-12-16, 2024-01-09] [2024-02-18, 2023-12-15, 2024-01-08, 2023-12-20, 2023-12-18, 2025-02-17, 2023-12-16, 2023-12-10] [2023-12-13, 2023-12-13, 2024-02-18, 2024-01-19, 2024-01-08, 2023-12-19, 2026-01-18, 2024-01-19, 2023-12-18, 2024-01-17] [] [] [2023-12-19 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [] [2023-12-14 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00] [2024-01-31 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00] [2024-01-17 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2015-01-25 00:00:00] [2024-01-19 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2024-08-03 13:08:30, 2023-12-13 00:00:00, 2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-16 00:00:00] [2024-01-19 00:00:00, 2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00] [2025-06-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00] +63 -798904965 \N -1 -888727596 2026-01-18 2025-06-18 2023-12-10 2024-08-03 - a so then really were I'm the will 有时候下来 q of - -- 1 1 52.087000 -1.200000 0.000000 40.123000 2023-12-12 00:00:00 2023-12-18 00:00:00 2023-12-09 00:00:00 2023-12-18 00:00:00 2026-01-18 00:00:00 2027-01-09 00:00:00 9999-12-31 23:59:59 2023-12-10 00:00:00 [0, 0, 1, 1, 1, 0, 0, 0, 1, 0] [0, 1, 0, 0, 0, 0, 1, 1, 0, 1] [] [] [] [52, -1, 0, -1, 0, 127, -128, 1] [] [] [-32061, 15740, 5564, 424, 9160] [-32768, 25817, 28583, 0, 135] [0, 32767, 32767] [284, 1, 12420, 0, 32767, -12579, -13426, 1, -1210, 32514] [147483648, 32679, -1] [-792519708, 147483648, -1, 0, 2076084511, 1689572909, -2147483648, 0, 0, 2147483647] [] [891946416, 2147483647, 1725987312, 147483648, 147483648] [147483648, 245, 0, 318880685, -733912764] [] [] [0, 9223372036854775807, 860611928275530726, 245, 147483648, 245, 9223372036854775807, -9223372036854775808, 147483648, 0] [1, -1, 5] [1429941565, 0, -1, 1, 7, 1073277835, -1, 0, 3, 0] [8, -2, -2, -1, -1, -1, 0, -1] [0, 1, 0, -1, -2, -65648596, 0, 0, -1, -2] [40.123000000000, -1.200000000000, 300.343000000000, 300.343000000000, 7.180300000000, -1.200000000000, 300.343000000000, -1.200000000000] [89.169900000000, 76.041400000000, 64.148500000000, -1.200000000000, 0.000000000000, 1.200000000000, 300.343000000000, -1.200000000000] [40.123000000000, 300.343000000000, 73.002000000000, 100.020000000000, 26.094500000000] [] [56.0437, -1.2000, -1.2000, 55.0694, 22.1782, 48.1660, 73.1698, 300.3430] [1.2000, 300.3430, -1.2000, 97.1391, 1.2000, 30.0571, 100.0200, 300.3430, -1.2000, 95.1952] [40.1230, 14.1951, 0.0000, 0.0000, 0.0000, -1.2000, 33.0662, 40.1230] [40.1230, 71.0967, 100.0200] \N \N [-1, 34, 40, 33, 1, 16, 35, 100] [100, 77, 100, 76, 29, 38, 37, 1, 18, 0] ["根本王晶深刻拐卖信阳黄河科学中原北商都不方便", "don't some like then didn't", "q", "are who to come", "been", "新亚非百货纱卡向你不太确定", "?", "往事"] \N [] ["u", "三星先达需方称呼", "or"] ["were", "all", "in"] ["过来", "", "what"] [] ["名字他在哪儿时文档小精灵做不到从你那里跟进天下无贼可是", "--", "had up you oh well did be"] ["g", "g", "y", "w", "f", "l", "p", "m"] ["q", "f", "v", "m", "d", "f", "u", "y"] ["n", "z", "f", "k", "t", "i", "c", "u", "b", "y"] [] [2024-08-03, 2024-02-18, 2027-01-16] [2023-12-11, 2027-01-16, 2027-01-16] [2023-12-17, 2023-12-13, 2017-04-20, 2023-12-11, 2026-02-18] [2025-06-18, 2024-02-18, 2024-07-01, 2023-01-15, 2024-02-18] [2026-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-16 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00] [9999-12-31 23:59:59, 2024-01-31 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2023-12-09 00:00:00, 2024-01-19 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00] [2023-12-16 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-07-01 00:00:00] [9999-12-31 23:59:59, 2024-01-08 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00] [2023-12-15 00:00:00, 2024-08-03 13:08:30, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00] [] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 2023-12-17 00:00:00] [2023-01-15 08:32:59.123123, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2025-06-18 00:00:00] [2023-12-15 00:00:00, 2023-01-15 08:32:59.123123, 2024-06-30 12:01:02.123000, 2023-12-16 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00] [2023-12-20 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] +64 -2120769389 -2147483648 773009967 32679 2023-12-18 2023-12-13 2023-01-15 2023-12-11 y e ? 你是谁 奖金 问题联创企业版员工 ? ? \N 0 -1.200000 0.000000 0.000000 300.343000 2024-02-18 00:00:00 2023-12-13 00:00:00 2026-02-18 00:00:00 2023-12-16 00:00:00 2024-01-17 00:00:00 2023-12-18 00:00:00 2023-12-18 00:00:00 2023-12-12 00:00:00 [1, 1, 0, 0, 0] [] [1, 0, 1, 1, 1, 1, 0, 0] [1, 0, 0] [2, -128, -90, 127, -128] [-128, -128, 4] [] [-1, 0, -116, 1, -128, 1, -113, 102, 119, 127] \N [-3629, -32768, 29780, 23443, -9543] [32767, 25124, -1, 0, 245, 32402, -1, 0] [-21892, 5012, -7480, 1, 1, -21362, 32729, 24919, 21001, 32767] [-75223152, -2147483648, -547517840, 2147483647, 2147483647, 557164758, 1, -1, -2048490891, 32679] [-1563953563, -1102398206, -164286524, -1, -1, 1, -2147483648, -477369972] [-1815509448, -929269037, 1845249612, -2147483648, -210587020, -423133792, -61635494, 866438444] [1796886689, 147483648, 1351391017, -1102548927, -1, 147483648, -1550591790, 814381028] [3994391313682659741, -9223372036854775808, 32679, 6414524256158555983, -1] [9223372036854775807, 3461318591445833402, 32679, -82834, -2650493, -5128746184055724428, -9223372036854775808, -1, 1, 1944873850] [-9223372036854775808, 9093140027280064153, 1642752, 9223372036854775807, 9223372036854775807] [] [-1, 0, 2006106710] [1, -2, 4, -758969412, 0, -1, -2, 1140112395, 1, 1] [0, 1, 1865162118, -783843214, 1, -1892982833, 1, 9] [1, 1, -2] [81.088900000000, 43.110300000000, 1.200000000000, 40.123000000000, 0.000000000000] [10.187700000000, 12.040400000000, 300.343000000000, 300.343000000000, 100.020000000000, 1.200000000000, 0.140000000000, 40.123000000000] [39.056500000000, 61.186900000000, 300.343000000000] [0.000000000000, 9.028100000000, 0.000000000000, 300.343000000000, 93.076300000000, 40.123000000000, 33.100200000000, 69.159100000000, 40.123000000000, 100.020000000000] [56.1674, 1.2000, 0.0000, 0.0000, 300.3430, 300.3430, -1.2000, 0.0000] [0.0000, 1.2000, -1.2000, 87.0422, 0.0000, 88.0070, 89.1350, 0.0000] [100.1167, 100.0200, 100.0200] [1.2000, 17.1833, 0.0000, 40.1230, -1.2000, 0.1914, 12.1520, 40.1230, 73.1419, 40.1230] [100, 20, 20, 100, 300, 1, 37, 100, 40, 54] [7, 0, -1] [] [40, -1, 55, 55, 0, 94, 300, 71] ["--", "go all you been this have up this", "look good see that had can't", "c", "-", "he's do but you I'll here as here", "性公司胖东来总天数处理一下说不清楚安华左右签字白菜", "some", "情人节", "后一个说不清楚我能不欢迎怎么摩擦洗衣服不在"] ["a", "-", "it's see for i get", "m", "埋在", "多家", "have", "they"] ["hey had one ok with it's say think his", "", "我在郑州满天飞长时间", "u", "to would it's so because I'll", "-", "was", "one", "I'm", "天马"] ["--", "m", "亨通这首治疗剑派只因为送货转账漏洞天天"] [] ["好早", "then ok", "--", "心中群号问好好处我打印暴力怎么打我海花一天", "郑娜"] ["-", "?", "v", "最好查证", "生日徐敬东而不是还以为同盟核算系列", "I'll my so", "小键盘", "?", "from will out some going some his go my", ""] ["-", "设置平志伟还在冲淡快运几乎连成卓兴之作", "had", "i", "", "me", "--", "无聊", "-", "里有张峰含税星星老同学一颗加载达成率系统集成原装"] ["y", "q", "q", "m", "v", "z", "m", "x", "p", "g"] ["e", "a", "l"] ["k", "n", "x", "a", "i"] ["q", "i", "f", "c", "t", "t", "a", "t"] [] [2025-06-18, 2025-06-18, 2024-06-30, 2023-12-13, 2024-02-18, 2024-01-08, 2023-12-10, 2025-02-17, 2024-01-31, 2023-12-18] [] [] [2023-12-12 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00, 2024-01-31 00:00:00] [2023-12-14 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00] [2023-12-13 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00] [2023-01-15 08:32:59, 2023-12-10 00:00:00, 2024-01-19 00:00:00] [2023-12-19 00:00:00, 2024-01-19 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00] [2026-01-18 00:00:00, 2023-12-14 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00] [2024-06-30 12:01:02.123000, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2012-06-28 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00] [2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-19 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-12 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00] [2027-01-09 00:00:00, 2027-01-16 00:00:00, 2027-01-16 00:00:00] [2024-01-08 00:00:00, 2024-01-19 00:00:00, 2027-01-09 00:00:00] [2023-12-17 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00] [2012-09-25 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2023-01-15 08:32:59.123123] +65 -1 1032031960 2147483647 -2147483648 2024-01-19 2024-08-03 2026-01-18 2023-12-19 高兴子夜煤炭备注爱人处理一下 we about or 小丽那样子招生哈哈小孩推荐这也是 want who ok see is all back well why what his were some - 1 0 \N 100.020000 0.000000 30.140000 2023-01-15 08:32:59.123000 2023-12-17 00:00:00 2023-12-18 00:00:00 2023-12-10 00:00:00 2024-02-18 00:00:00 2025-02-18 00:00:00 2024-01-09 00:00:00 2027-01-09 00:00:00 [1, 1, 1, 0, 1, 1, 0, 0] [] [] [1, 1, 0, 0, 0, 1, 0, 0, 0, 1] \N [9, -85, 6, 127, 1] [] [101, -61, -1, -128, -1, 1, 7, -128] \N [5756, 1, 32767] [] [-10604, 245, -12650] [221582721, -1, -2147483648] [1263566679, -72690303, -2147483648] [-2147483648, 0, 1, 32679, 1] [1, -446557633, 985055142] [-1, 9223372036854775807, 0, -695856552144840969, -9223372036854775808] [] [147483648, -3018069284104613275, -9223372036854775808] [32679, 32679, 5036236908060804190, 0, -4613643] [-2, 1, 0, 1, 8] [] [1, 1, 1, 0, -1, 3, -1283361846, 1, 6, -2] [-1, 8, -2, -2, 0, -2034611564, -567802713, 8, 0, -2145338996] [1.200000000000, 0.000000000000, 40.123000000000, 100.020000000000, -1.200000000000] [1.200000000000, 58.093400000000, 31.019800000000] [1.200000000000, 1.200000000000, 30.018500000000] [] \N [1.2000, 89.1254, -1.2000, 0.0000, 80.0107] [100.0200, 98.1223, 9.0284, 42.0022, 52.1504, 300.3430, 1.2000, 0.0000, 83.0775, 26.1436] [0.0000, 94.1805, 3.1295, 36.1449, -1.2000] [0, 40, 1, 300, 51, 84, -1, 65, 31, 300] [10, 28, 1, 1, -1, 1, -1, 0] [-1, -1, 300, 300, 68] [0, -1, 0, 39, 49, 80, 300, -1] [] ["研究院", "也得", "up do be here that", "有缘", "a", "-", "one with would the will is up", "the did time and yeah have"] ["通信全程锐康几次信用社", "--", "?"] ["back", "郭晓见过面", "he's why so there go didn't go go", "", "all", "你来找我吧提示", "you're", "看出来士大夫金立四天键盘群号经销总代理董事长为难", "", ""] ["?", "be at one like right me yes was hey as", "微波", "", "提供"] ["新亚非太不导入走在视野证书相见不下", "l", "?", "no there go get get this got", "", "the I'm all why why she and him be going", "to", "蓝色差不多世安掉线派人刻意工作还行", "but", "曾经见到李东"] ["", "", "-", "你这个别人", "--"] [] \N ["b", "u", "d", "r", "c", "h", "x", "n"] ["k", "k", "g"] ["q", "m", "i", "o", "k", "p", "i", "i"] [2024-08-03, 2024-02-18, 2023-12-16, 2023-12-19, 2023-12-14, 2026-01-18, 2025-06-18, 2011-11-28] \N [2024-01-17, 2023-12-13, 2024-01-09] [2023-12-10, 2025-02-17, 2023-12-13, 2024-02-18, 2023-12-17] [2027-01-16 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2003-02-09 00:00:00, 2025-02-17 00:00:00, 2014-05-23 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00] \N [2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00] [2023-12-18 00:00:00, 2024-01-19 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00] [2023-12-10 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2025-02-17 00:00:00] [2024-02-18 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00] [9999-12-31 00:00:00, 2023-12-14 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00] [2024-01-08 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00] [2023-12-17 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00] [] [2007-09-01 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00] +67 \N -2147483648 1 147483648 2024-01-19 2023-12-14 2025-06-18 2024-01-31 外运蓝色若虚通讯找不到神仙海尔老公有关麻烦 v going no he's about there ? what when been got have this can i back my if you he's see just ? 儿时 0 1 40.123000 47.178900 0.000000 100.020000 \N 2023-12-09 00:00:00 2023-12-14 00:00:00 2024-01-08 00:00:00 2023-12-20 00:00:00 2027-01-09 00:00:00 2023-12-18 00:00:00 2023-12-18 00:00:00 [0, 0, 0, 1, 0, 0, 0, 1, 1, 0] \N [1, 0, 0] [1, 1, 0, 1, 0, 1, 0, 0] [0, 127, 4, 1, 1] [30, -6, 127] [-110, 65, 4, 1, -1, 1, 1, -128, 1, -117] [0, -128, -99, -34, 0, -74, -1, 13, 0, 119] [21854, 2115, 32767] [-26623, 245, 32767, -3938, 1, 0, 9819, 2702] [-14432, 6899, 32767, 14585, -1] [] [] [] [] [-1957207881, -1827267830, -1881235619, 1, -469225401, -1085587456, 0, -52335484, 702424776, 1489129289] [-2980664974152692130, -6566136, 147483648, 7072718851149334424, 32679] [32679, -1, 224228431610272525, 245, -9223372036854775808] [32679, 2465874, 1, -6706335527102299087, -9223372036854775808, -1874236312899441296, 147483648, 1, 245, -1] [245, -2144919699, 3309831, 147483648, -1] [1785186770, 1, 6, 1, -2] [0, 0, 4, -1, 9, -2, 1, 1] [-1, 8, -399012959, -2, -1] [] \N [40.123000000000, 1.200000000000, -1.200000000000, 40.123000000000, 0.000000000000, 71.026400000000, 100.020000000000, 40.123000000000, 0.000000000000, 1.200000000000] [40.123000000000, 1.200000000000, 6.166100000000, 1.200000000000, 100.020000000000, 75.192500000000, -1.200000000000, 100.020000000000, -1.200000000000, -1.200000000000] [8.109800000000, 40.123000000000, 23.057300000000] [300.3430, 1.2000, 300.3430, 100.0200, 300.3430, 40.1230, 300.3430, 40.1230] [1.2000, 100.0200, 100.0200, 40.1230, 300.3430, 0.0000, 96.1642, -1.2000, 14.1873, 40.1230] [35.1909, 100.0200, 100.0200, 0.0000, 100.0200, 59.0409, 100.0200, 0.0000, 57.0162, -1.2000] [5.0691, -1.2000, 63.0249] [] [40, 100, 0] [-1, 95, 66, -1, -1] [100, 1, 20, 300, 40, 100, 33, 1, 68, 100] ["文艺", "I'm in him", "愿意王佩丽东创修订企业内部", "-", "进来", "过节费磁带机", "p", "去不了"] ["?", "耗材拜托明威添加眼睁睁自动化视野", "can't"] ["really up are so would got don't they time", "she a can have do time", "成交", "got", "祺祥你家难缠王星大键盘不够看法需要帮忙计划市场"] [] ["o", "朱晓明不分恢复下次反对首位大风检验通用", "mean"] ["熊朝华理光长信都还问好", "?", "know your tell will have on", "", "I'm"] ["?", "-", "my"] ["--", "没时间", "", "意见需方怎么了群号情愿表示难道区号热卖", "you're"] ["z", "m", "y", "a", "r"] ["u", "m", "m", "y", "j"] [] ["f", "b", "j"] [2023-12-10, 2024-06-30, 2025-02-18, 2023-12-15, 2026-01-18, 2023-12-14, 2014-08-12, 2024-01-17] \N [] [] [9999-12-31 23:59:59, 2024-06-30 12:01:02, 2026-01-18 00:00:00, 2024-06-30 12:01:02, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00] [2023-12-16 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00] [] [2023-12-13 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00] [2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 2024-01-19 00:00:00] [] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2024-01-08 00:00:00] [] [2023-12-09 00:00:00, 2023-12-16 00:00:00, 2023-12-20 00:00:00] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-09 00:00:00, 2023-12-18 00:00:00] [] +68 2147483647 1 0 -2107971368 2024-02-18 2024-01-17 2024-01-31 2023-12-12 do see here how there some something me what in something got as be from if really 聊过备用金权威忽视 最近忙什么呢立博你吃法 生命中委托收款热销孙健也是曙光厦门王庄乡 so so ? 1 1 1.200000 100.020000 32.156700 -1.200000 2027-01-09 00:00:00 2005-08-28 00:00:00 2023-12-20 00:00:00 2024-01-17 00:00:00 2023-12-16 00:00:00 2025-06-18 00:00:00 2023-12-18 00:00:00 2023-12-16 00:00:00 [1, 1, 1, 1, 0, 0, 0, 1, 0, 0] [1, 0, 0, 0, 1, 1, 1, 1, 1, 0] [] [1, 1, 0, 1, 0, 1, 1, 0, 1, 0] [4, 3, -1, 0, 0] [0, -128, 127, 1, -38] [-109, 0, -128, 9, 6] [-128, 3, -1] [245, -6592, 245, 22646, 17527, 0, 32767, -32768, 32767, -1] [-16966, 0, 16613, 29327, -32768] [3623, -15826, -1, 1451, -1, -2241, 16210, 14385, 245, 245] [781, 24383, 245, 245, 5829, 15406, -10514, 32767, 23980, -30490] [0, 2147483647, -1748979184, 1262223470, -576335723, 32679, 32679, -1] \N [1240809143, -955309478, 1724645396, -1104215797, -1, -1, 1, -2147483648, 0, -769490769] [32679, 1412993917, 32679, 1015557134, -2147483648, -2147483648, 795469098, -470754718] [-9223372036854775808, 245, -267770528, 9223372036854775807, 5214560714115424127] \N [] [] [-1, 6, 0, -1, 1, 3, 3, 3, -2, 0] [-2, -1, 501256695] [] [2, -2, 7, 886798110, 0, 0, -2, 809227662, 1, 0] [0.000000000000, 40.123000000000, 300.343000000000, 300.343000000000, 100.020000000000] [0.000000000000, 82.050500000000, -1.200000000000, 3.195500000000, 40.123000000000, -1.200000000000, 92.177300000000, 57.045300000000, 58.174700000000, 44.184400000000] [58.080200000000, -1.200000000000, 90.188600000000] [68.189700000000, 84.037600000000, 47.026000000000] \N [100.0200, 40.1230, 98.1909] [34.1039, 100.0200, 18.1785, 49.1152, 0.0000, 300.3430, -1.2000, 40.1230] [56.0684, 0.0000, 300.3430, -1.2000, 300.3430, 100.0200, -1.2000, 300.3430, 300.3430, 1.2000] [45, 0, 1, 0, 300] [40, 33, 100, 27, -1, 23, 67, 19, 40, 40] [-1, 300, 40, 40, 40, 1, 0, 2] [24, 0, 1, 300, 1, 69, 64, 100] ["y", "think from come her come it she", "see now ok there from is be", "回忆", "h"] ["-", "-", "?", "张姐数据哦龙飞卓兴生意经过还跟他包装行业", "see"] ["not", "I'll well me and can was when", "s", "--", "-"] [] ["", "of go then I'm hey this just up could do", "那鱼加密狗送人新国"] ["n", "随其自然高端信息照片而不是煤炭", ""] ["had", "?", "手机吧", "going with mean for would now", "--", "know hey we his her going in back", "毕业墙发吧给我晚上聊翔龙喝酒早上小刘就是", "that's", "r", "just"] [] ["a", "s", "c", "m", "v", "y", "n", "k", "z", "k"] ["p", "q", "j", "r", "e", "j", "l", "l"] ["n", "c", "q"] ["m", "z", "f", "w", "f", "w", "n", "i", "s", "w"] [2023-12-16, 2024-02-18, 2023-12-20, 2025-02-18, 9999-12-31, 2024-02-18, 2026-01-18, 2024-01-17, 2023-12-18, 9999-12-31] [2023-12-17, 2024-08-03, 2023-01-15] [2023-12-15, 2018-12-13, 9999-12-31, 2024-01-19, 2023-12-12] [2024-01-08, 2024-01-08, 2024-07-01, 2024-01-19, 2025-02-17] [2023-12-10 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 9999-12-31 23:59:59] [] [2024-01-19 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2023-12-16 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00] [2024-01-31 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2027-01-09 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 2024-07-01 00:00:00] [2014-08-12 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00] [2023-12-16 00:00:00, 2024-08-03 13:08:30, 2023-12-14 00:00:00] [2023-12-16 00:00:00, 2026-01-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-12 00:00:00] [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [] [9999-12-31 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59] +70 1518025138 1 32679 -1 2026-02-18 2014-08-12 2023-12-20 2024-01-17 刚刚 f \N -- oh yes why that why want mean right yeah yes know you're who this going there a that's see 1 1 1.200000 300.343000 59.003800 -1.200000 2023-12-15 00:00:00 2024-02-18 00:00:00 2024-06-30 12:01:02.123000 2027-01-09 00:00:00 2024-01-08 00:00:00 2024-08-03 13:08:30 2024-01-31 00:00:00 2023-12-09 00:00:00 [0, 1, 1] \N [0, 1, 1, 0, 0, 1, 0, 0, 0, 0] [0, 0, 1, 0, 0] [46, 1, 1, 127, 113] [-1, -128, 1, 127, 8] [83, -45, -128, 109, -124] [-128, 126, 119, 4, -128, -1, -128, 127, 1, 6] [-4971, -1, -32768] [18927, 0, -13589] [] [13554, 0, 736, 1, 1, 14217, 0, -10024, -32768, 22754] [-107026164, -733887695, 0, -2147483648, 32679, 1422155598, 32679, 32679] [] [] [] [245, -7073936091063573821, 32679, 32679, 1175206, 0, 2084634048245570251, 1] \N [] [] [-1, -2, -1388026484, 1, -1] [0, 0, -1635556876] [] [] [100.020000000000, 0.000000000000, 100.020000000000, 40.123000000000, 1.200000000000, 100.020000000000, 93.195200000000, 53.042900000000] [-1.200000000000, 0.000000000000, 32.119200000000] [] [] [98.0777, 4.1290, 1.2000] [] [40.1230, 100.0200, 100.0200, 88.1528, 1.2000, 40.1230, -1.2000, -1.2000] [-1.2000, 23.0182, 0.0000] [-1, -1, 0, -1, 100, 40, -1, 40, 40, 300] [1, 50, 0, 37, 17, 300, 40, 0] [300, 1, 16, 94, 28, 300, 1, 87] [100, -1, 300] ["注册", "because yeah going in come yeah for yes", "you then yeah", "o", "平志伟", "", "well", "进入"] \N ["--", "", "had was say because her going up of time did", "晶咪旭日下载惊喜下半年金总核心这家", "just not ok not"] ["or", "", "look something can right the and look from or", "实话口语文杰趋向性礼拜六道歉相约属于你又是陪着", "一两", "", "", "娃娃"] ["尤其", "-", "?", "we", "--", "p", "-", "?", "he", "me think out was for could when want"] ["?", "right", "--", "some could mean can will mean", "库房"] ["", "what", "", "几台", ""] ["got", "运费", "your", "失误宠物小浪底", "--"] ["t", "f", "w"] ["u", "d", "z", "f", "d", "i", "r", "v", "e", "p"] ["l", "s", "w"] ["k", "t", "u"] [2023-12-15, 2023-12-20, 2023-12-17, 9999-12-31, 2023-12-16, 2026-01-18, 2023-12-12, 2023-12-20, 2024-01-09, 2023-12-17] [2023-12-14, 2023-12-12, 2023-12-10, 2023-12-10, 2024-08-03] [2000-09-08, 2014-08-12, 2023-12-14, 2027-01-09, 2024-01-08, 2025-06-18, 2023-12-09, 2023-12-10] [2027-01-16, 2024-07-01, 2023-01-15, 2024-01-17, 2024-08-03, 2025-02-17, 2024-08-03, 2023-12-15, 2025-02-17, 2014-08-12] [2024-07-01 00:00:00, 2023-01-15 08:32:59, 2023-12-09 00:00:00] [2027-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00] [2026-01-18 00:00:00, 2023-12-13 00:00:00, 2024-01-31 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00] [] [2023-12-15 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00] [2024-02-18 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59] [2024-02-18 00:00:00, 2025-06-18 00:00:00, 2005-02-11 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00] [2023-12-09 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59.123000] [2026-02-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00] [2025-02-17 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2027-01-09 00:00:00] [2023-12-11 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00] [2014-08-12 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2013-05-01 00:00:00, 2025-06-18 00:00:00] +71 0 1 -1223066739 1924771214 2003-04-13 2023-12-20 2023-12-09 2023-12-14 ? 发送 it see why about the who she back this 工资 a about like did for time did that -- -- 1 1 \N 47.041700 100.020000 1.200000 2024-01-17 00:00:00 2026-02-18 00:00:00 2024-06-30 12:01:02.123000 2027-01-09 00:00:00 2024-01-19 00:00:00 2024-02-18 00:00:00 2025-02-17 00:00:00 2024-08-03 13:08:30 [0, 1, 1, 0, 0] [0, 1, 1, 0, 0] [1, 0, 0, 1, 0, 0, 1, 0, 0, 1] [1, 0, 1, 0, 0] [] [-128, 0, -1, 13, -29] [94, -1, -128, 50, -1] [67, 11, -90] \N [-9464, -19859, 1, -21868, -3911] [-10646, -15465, -32768, 2924, -4294, -6400, -1, -32512] [-1, 0, 1, -1, 32767, 23474, 1, 245] [-1318451995, -1307573547, 47625331, -363171195, 32679, 2147483647, 1, -1] [-1712865700, 32679, -1] [32679, -2115915422, -1214339356, 2147483647, 782589591, -2147483648, 32679, -1, 32679, -1123192583] [1557474205, 32679, 1091072990, 1681949343, 0] [245, -2570639851567583390, 147483648, -1, 1068346875899518035, 32679, -3664535, 0, 9223372036854775807, 4938034754146884664] [147483648, -2131501229, 9223372036854775807, 147483648, -1676609932, -9223372036854775808, 245, -9223372036854775808] [9223372036854775807, 6739195542281651758, -9223372036854775808, 2880170980805648627, 1334742560386411140, 9223372036854775807, 9223372036854775807, 0, -5019958216744415083, 32679] [1, 147483648, 5253598448919332689, 147483648, 32679] \N [0, 6, -2, -1, 1, -2, 0, -2, 742810188, -1] [0, 1, 1, -1, 0] [0, -1, 4, 1, 0, 0, -2, 5, 1, 0] [0.000000000000, 0.000000000000, 74.160900000000, 40.123000000000, 40.123000000000] [91.140800000000, 300.343000000000, 69.042300000000, 76.012900000000, 300.343000000000] [1.200000000000, 72.082400000000, -1.200000000000, 300.343000000000, 0.000000000000, 100.020000000000, 23.192100000000, 52.035800000000, 300.343000000000, 85.063100000000] [40.123000000000, 0.000000000000, -1.200000000000, 0.000000000000, 1.200000000000] [66.0064, 67.0305, 1.2000] [100.0200, 92.1517, 1.2000, 1.2000, 100.0200, 65.0277, 1.2000, 300.3430, 10.0721, 100.0200] [87.0140, 1.2000, 71.1392] [-1.2000, 7.1571, 300.3430, 98.0750, 100.0200, 40.1230, 40.1230, 300.3430, 5.0689, 100.0200] [86, 1, 40, 40, 100, 84, 40, 1, 1, 24] [] [] [22, 97, 24, 60, 65, 36, -1, 300, -1, 62] ["a", "that's", "yes"] ["", "美捷", "才到众诚刘亮讲师提供美好看不到"] ["五万多条还有呢核心底下众人刑阳股份有限公司角度", "yes your now", "--", "请到双机走过大家好一小本就主机太多算数", "未税摆放治好还没见计算开除", "没变瑞和直销", "", "刘雪"] ["--", "is did", "吹去", "在不在", "can't have think you no time that of there", "第六待定等着文艺中的", "--", "all", "通讯简历内训通讯蠢材", "okay"] ["-", "--", "could", "", "小徐"] ["at", "他在哪", "不信明显金汇通晚上聊公司元旦所谓", "-", "吃住", "--", "", "?"] ["?", "", "新乡市最大化来了参考", "信天游", "高一些及其环保局", "v", "有空吗", "-", "制胜副本看完偶然", "倒掉"] ["--", "", "一岁"] ["j", "i", "g", "q", "x", "p", "i", "p", "c", "w"] ["w", "i", "w", "k", "g", "i", "f", "w", "f", "t"] ["m", "r", "x", "e", "g", "q", "m", "k", "p", "l"] ["n", "o", "h", "k", "n"] [2024-01-17, 2023-12-12, 2024-08-03, 2025-06-18, 2023-12-11, 2024-01-17, 2024-01-09, 9999-12-31] \N [2023-12-20, 2024-02-18, 2023-12-12] [2027-01-16, 2026-02-18, 2027-01-16, 2025-06-18, 2023-12-10] [2024-01-17 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02, 9999-12-31 23:59:59] [2002-04-27 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00] [2027-01-16 00:00:00, 2023-01-15 08:32:59, 2024-01-31 00:00:00] [2016-02-22 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00] \N \N [] [2023-12-12 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00] [2026-02-18 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00] [2023-12-10 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00] [2026-02-18 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00] [2024-08-03 13:08:30, 2023-12-17 00:00:00, 2024-01-31 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2027-01-16 00:00:00, 2025-06-18 00:00:00] +72 -1 1430407156 -1 1481688972 2024-02-18 2025-06-18 2027-01-16 2024-07-01 this about did his - 含义不会被濮阳 that about the who 龙飞 关机拜托方面咱用元转到在你是电子国栋 -- 1 0 91.122500 83.159000 42.091300 1.200000 2023-12-20 00:00:00 2024-07-01 00:00:00 2023-12-13 00:00:00 2025-06-18 00:00:00 2023-12-15 00:00:00 2025-06-18 00:00:00 2023-12-15 00:00:00 9999-12-31 00:00:00 [] [0, 1, 0, 0, 0, 1, 0, 1] [1, 0, 1, 0, 1] [1, 1, 0, 0, 0, 0, 0, 0] [50, -60, 2] [127, 1, 4, -87, 1] [-1, 1, 1, 1, -69, 1, -121, -1, 32, -1] [0, 5, -65] [] [23665, 32767, -30251, -32768, -32768] [] [1, 1, 12161, -32768, 245, 245, 17153, -15965, 9029, 10875] [1955190392, 147483648, -368044409, -1284499679, 1087170724, -1, 147483648, -2147483648, -2147483648, 332237054] [] [] [0, 2147483647, -2147483648] [-1, -9223372036854775808, 9223372036854775807, -9223372036854775808, 2578436833245066946, 147483648, -1, 1, -1, 1] \N [5111334588438205011, 1, 1381634601, 32679, 7401184405260009108, -7721583, 147483648, -9223372036854775808, 147483648, 1] [245, -7241483, -9214720223173013728, 32679, -8533187697347224777, 245, -7909302, 147483648] [5, 0, -2, -2, 0] \N [1, -1, 1, 0, -918886667] [-1, 0, -2, -1427547705, 1, -2, -2, -895595928] [12.089600000000, 40.123000000000, -1.200000000000] [72.171700000000, 100.020000000000, 32.041700000000, 1.200000000000, -1.200000000000] [100.020000000000, 300.343000000000, 20.049900000000, 100.020000000000, 0.000000000000, 37.192700000000, 40.123000000000, 61.046100000000] [] [81.0093, 40.1230, -1.2000, 40.1230, 40.1230] \N [] [-1.2000, 12.1306, 61.1472, 13.0259, -1.2000] [64, 0, -1, 300, 40] [300, 100, 40] [] [50, 40, 89, 1, -1, 48, -1, 0, 40, 40] \N ["?", "that's that's just they who in you're can't good want", "or be at on you", "?", "", "打扰", "附注", "right"] ["过了", "oh think it get been a", "-"] ["-", "", "see because", "前奏", "just not that"] ["一种", "going", "五月份", "--", "去做消息文华", "p", "彩页吹去你杀时间处理完", "商家不止有空名字顺河路全勤奖群管理员另存为追到插拔"] [] ["?", "here", "爱情终究实际信源语音长期第二个"] [] \N ["f", "m", "q", "f", "r", "r", "f", "a", "o", "g"] ["t", "b", "f"] ["n", "x", "o", "f", "e"] [2023-12-12, 2024-01-19, 2024-02-18, 2024-02-18, 9999-12-31, 2026-02-18, 9999-12-31, 2023-12-17] [2027-01-09, 2024-01-17, 2023-12-19] [2024-07-01, 2024-02-18, 2023-12-20, 2026-02-18, 2023-12-18, 2024-07-01, 2025-06-18, 2027-01-09, 2024-01-31, 2027-01-16] [] [2024-08-03 13:08:30, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00] [2023-12-11 00:00:00, 2002-08-25 00:00:00, 2027-01-16 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00] [2023-12-18 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00] [2024-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-19 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2023-12-14 00:00:00] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123000, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2023-12-17 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00] [2024-07-01 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-16 00:00:00, 2023-12-19 00:00:00] [2023-12-09 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00] [2023-01-15 08:32:59.123123, 2023-12-17 00:00:00, 2014-09-06 00:00:00, 2024-01-17 00:00:00, 2023-12-14 00:00:00] +73 1739090754 -676882685 876200412 -1 2023-12-19 2024-07-01 2024-01-31 2024-08-03 who \N - he you good it you're yeah 金总名称下雪显卡士大夫问了品放连接线人数你呢岂能 know know come to back didn't there when her ? 就算 0 1 300.343000 100.020000 100.020000 0.000000 2026-02-18 00:00:00 2024-02-18 00:00:00 2025-02-17 00:00:00 2023-01-15 08:32:59.123000 2024-06-30 12:01:02.123000 9999-12-31 23:59:59 2014-08-12 00:00:00 2023-12-14 00:00:00 [0, 1, 0, 1, 0, 1, 1, 1] [0, 1, 0, 0, 0, 0, 1, 0] [] [] [29, 0, -128, 0, 56, 0, -128, 27] [-128, 1, -25, 0, 0] [1, -10, -128] [5, 127, 127, 1, 127, -1, 0, -1] [0, 679, -32768] [] [] [] [-1, 869680128, -1] [-1533314376, 935970542, 1283380926] [1, 32679, 1322864531] [981248645, -2147483648, 32679, 1845956973, 2116095765, 2147483647, 0, 1477361225] [1, 32679, 0] [] [245, 237953767335188445, 6943652, -5964837438026320587, 147483648, 245, 1, -760152246071120508, -7097834372236152415, 6686922770139763888] [147483648, 0, 9223372036854775807] [6, 1, -2, 1, 0, -1, 1, -2, -1, 0] [-1, -1, -1583269299, -2, 0] [-1, 0, 3] [-2, 1, -2] [] [40.123000000000, 0.000000000000, 300.343000000000, 100.020000000000, 0.000000000000] [86.042800000000, 300.343000000000, 100.020000000000, 85.067900000000, 40.123000000000, 100.020000000000, 34.091600000000, 100.020000000000, 100.020000000000, 300.343000000000] [300.343000000000, 1.200000000000, 47.120000000000, 77.133000000000, 61.104300000000] \N [1.2000, 0.1460, 93.1465] [0.0000, 40.1230, 79.1069, 300.3430, 11.1606] [100.0200, 1.2000, 100.0200, 1.2000, 21.0021, 0.0000, 11.0686, 100.0200, 1.2000, 100.0200] [] [16, -1, 78, -1, 300] [] [300, 40, 1, -1, 40] ["晓燕", "one", "太远", "科技", "-", "欢迎欢迎郑州市我在同学家七匹狼帮不了你五十二万签订注册表只能代表", "?", "扩展名", "不传", "飞鸽向西"] ["b", "did", "k", "c", "all", "me", "-", "", "降低缺货一百万高级总代理明白线上病毒好不好诚心彩虹", "all"] ["想要", "创新离职不错儿童诚信休息最近忙什么呢感受不但是", "都发生今收到快车详谈运输创新人往高处普通夜间亮度", "金达绿城动画昨天会想申腾刀片服务器", "投缘", "o", "can", "双子", "最好张丽萍马上", ""] ["", "王总签不签都", "一点", "-", "煤气地市支原体催催差不多报表农业万邦联想"] ["for", "状态大中华字库文件配件怎么找顶多要不然啥时回来", "光缆", "it's oh to just going don't", "she", "--", "we just in that", "肥姐南三环定金长江路结合谁家小雪诊断年轻不走"] ["做出", "我还", "-", "didn't", "公里"] ["关机", "辅佐", "刚刚出去纯平气人没变物流沐浴一共"] ["z", "--", "--"] ["h", "e", "j", "u", "j", "p", "n", "l"] \N ["c", "x", "n", "d", "v", "r", "g", "s", "j", "v"] [] [2024-01-17, 2027-01-16, 2025-02-17, 9999-12-31, 2024-01-08, 2023-01-15, 2024-01-09, 9999-12-31] [2024-02-18, 2011-10-06, 2023-12-20, 2023-12-13, 2023-12-14, 2024-01-31, 2026-02-18, 2023-01-15, 2025-02-18, 2025-06-18] [2024-01-31, 2023-12-20, 2023-12-13] [2023-12-20, 2024-02-18, 2023-12-20] [2027-01-16 00:00:00, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2024-02-18 00:00:00, 2007-07-12 00:00:00] [2024-07-01 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00] [2023-12-12 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59, 2025-02-17 00:00:00, 2027-01-16 00:00:00] [2024-01-19 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00] [2024-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59.123000] [2026-01-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2023-12-18 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00] [2014-08-12 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00] [2024-01-31 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00] [2023-12-17 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00] [2014-08-12 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00] +74 1463183787 -2147483648 1121730538 -1 2023-12-10 2023-12-15 2023-12-16 2023-01-15 ? see \N 办完倚天参数表调货杀客折旧史丽 那就 青云出现拟在建爱帐期纽曼 ok see my to you're there going if some want okay just that from he's his \N 0 47.120400 1.200000 100.020000 1.200000 9999-12-31 00:00:00 2023-12-19 00:00:00 2023-12-18 00:00:00 9999-12-31 23:59:59 2024-01-09 00:00:00 2023-12-12 00:00:00 2023-12-13 00:00:00 2026-01-18 00:00:00 [0, 1, 1] [1, 1, 1, 1, 0] [1, 1, 0, 0, 1, 0, 1, 1, 0, 1] [1, 1, 0, 1, 1, 0, 0, 0, 1, 0] [1, 127, -128, 12, -128] [-46, -101, 127, 1, 9, 3, 6, -128, -128, 101] [] [4, 75, 4, 6, 0] [] [28171, 12305, -5547, -1754, -19569, 245, -5464, 165, 5656, 0] [] [32767, -1, 26795, 2206, -32768, -1319, 0, 10721] [1, 0, 2147483647, 0, -771756633] [-805109252, 0, 1, 32679, -1137614723, -438777665, -220949627, 1132169560, 147483648, -1] [-893080308, -102874450, -1128023527, -1986253592, 1, -1634235053, 2147483647, 147483648] [-1674195833, -687165297, -1] [32679, -5738326049851237503, 0, 32679, 32679, 147483648, -2125339959942565234, -5421900191729472429] [147483648, 1, 245] [0, -8164500083403885273, -1, 147483648, 2995811621896510115, 147483648, -1, 1] [-9223372036854775808, -9223372036854775808, 7999026, 6909398, -1] [1888934442, 781520776, 0] [-2, 0, 0, 0, -2, 0, -2, -154507872, 1, 1] [1588120707, -1, 5] [-1, 0, 1538446394, -1, 0, 1, 1, 1360640907, -279050615, -2] \N \N [0.000000000000, 0.000000000000, 100.020000000000, 1.200000000000, 55.015600000000] [0.000000000000, 100.020000000000, 0.000000000000, 42.014900000000, 100.020000000000, 0.000000000000, 300.343000000000, -1.200000000000, 40.123000000000, 100.020000000000] \N [0.0000, 9.1954, 100.0200] [0.0000, 88.0658, 0.0000, 10.0675, 40.1230, 74.0372, -1.2000, 1.2000, 71.1375, 1.2000] [300.3430, 56.1844, 300.3430, 300.3430, 61.1160, 40.1230, -1.2000, 48.0752, 15.1803, 100.0200] [-1, 29, 300, 40, 2, -1, 300, 71, 40, 1] [100, 52, 16, 40, 89] [1, 300, 73, 97, 24, -1, 33, 40, 76, 100] [0, 34, 75] ["早上", "", "然后", "--", "they up be get yeah out he's", "小徐忘记不了你问火球代表实现词语我就不要错", "a", "for", "y", "it's as the as with him going up"] \N [] ["her", "-", "the have got go did", "?", "-", "-", "--", "休息那是说不清楚有误"] ["?", "-", "could the but want will think really it's how at", "she got that to at out", "后者意见", "去享受驱动器东风系列一脚光靠浪潮你早几天回来吧", "come", "f", "请他原厂合作过会议时机老是汉威", "感觉"] ["", "-", "about something there look"] ["d", "魏军出入相逢", "?"] [] [] ["x", "g", "s", "s", "q"] ["u", "m", "z", "e", "q"] ["c", "m", "k", "z", "y"] [2023-12-16, 2026-01-18, 2024-02-18, 9999-12-31, 2023-12-13] [2024-01-19, 2024-01-19, 2024-06-30, 2023-01-15, 2024-01-17] [2023-12-12, 2023-12-15, 2014-08-12, 9999-12-31, 2024-02-18, 2023-12-17, 2024-02-18, 2024-07-01] [2027-01-09, 2023-12-12, 2023-12-15, 2024-01-08, 2023-12-20] \N [2024-02-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00] [2024-07-01 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00, 2023-12-11 00:00:00] [2027-01-16 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00] \N [2023-12-16 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2027-01-09 00:00:00] [2017-10-08 00:00:00, 2023-01-15 08:32:59.123000, 2024-08-03 13:08:30] [2023-12-15 00:00:00, 2016-07-22 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00] [2023-12-14 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2026-01-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00] [2025-06-18 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2009-09-04 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2023-12-09 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-11 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00] +75 1 32679 1322684238 -1841659783 2026-02-18 2025-02-17 2024-07-01 2023-12-13 正常 l 技术 ? 院内有限公司兰州妹妹资讯看待盗帅 兼得接触告诉 经营有带你浇水网通史丽 \N 0 41.077200 7.187000 59.099700 100.020000 2023-12-15 00:00:00 \N 2023-12-09 00:00:00 2023-12-19 00:00:00 2024-07-01 00:00:00 2024-06-30 12:01:02.123000 2027-01-09 00:00:00 2023-12-17 00:00:00 [0, 0, 0] [] [1, 1, 0, 1, 0, 0, 0, 1] [0, 1, 0, 1, 0, 0, 0, 0, 0, 0] [-1, -109, -1, 127, -128, -101, -91, -128, -1, 127] [127, 37, 0, 1, -86, 0, 0, 127, 1, 0] [86, 0, -1, 71, -40] [-1, -46, 1, -100, 0, 6, -1, 1, 9, 2] \N [20725, -31531, -28451, 1, -1, 13060, -1, 32767] [-4452, 11937, 20942, 28604, 3259, -1, 0, 245, 29993, 1407] [] [-1196241366, -1, 1, 0, -1042498042, 775655558, 2028422091, -1] [147483648, 619734674, 1563977770, 85107561, -2147483648, -1, 1, 1] [2147483647, -944564912, 2147483647] [147483648, 32679, 2037636336, -1, 1487597306] [5988431458804567656, 8749500353419892461, -9223372036854775808, 32679, -9223372036854775808] [147483648, 32679, -265249, 5337062949083451779, -2063128110509931652, 3185592307430713676, 245, 1] [116057, -3807377795150931569, 491392881215848826] [-1142267772, 1, 1852130243] \N [1970394636, 2, -2] [-1329202722, 0, -2, -1, 0] [-1, -1, -2, 1, -1, 0, 0, 0] \N [] [] [300.343000000000, -1.200000000000, 54.199200000000, 1.200000000000, 100.020000000000, 40.123000000000, 66.096900000000, 0.000000000000, 300.343000000000, 40.050400000000] [38.1857, 300.3430, 0.0000, 0.0000, 300.3430, 38.1788, 12.1930, 40.1230] [300.3430, 56.1427, 100.0200, 25.0283, 1.2000] [0.0000, 50.1076, 300.3430] [79.1108, 58.0779, 40.1230] [62, -1, 1, 29, 0, 49, 98, 40, 4, 100] \N [43, 40, 46, 40, 7, 90, 39, 68, 1, 97] [40, 37, 300, 0, 4, 300, 1, 40, 73, 100] [] ["氛围", "家里崔广为火车站做生意直销年轻铁路有波", "s", "会员卡", "well have him"] ["up", "they", "-", "日益交代不到无聊字库文件那种玫瑰留一下阵列智慧", "she", "c", "没钱汇祥简历", "in look", "?", "-"] [] [] ["所要求", "him that he her is want", "ok yes say could that you got did"] ["没有货资金支票到货申请人切换器任务", "王文非看法底价河南总代理大山报表新增", "广告词", "me who will", "-"] [] [] ["a", "f", "f", "q", "a", "l", "l", "t"] ["v", "i", "a", "t", "b", "a", "i", "s"] ["u", "l", "k", "j", "q", "e", "q", "q", "l", "v"] [2023-12-12, 2023-12-15, 2023-12-14, 2024-08-03, 2023-12-09, 2023-12-19, 2024-01-19, 2026-01-18, 2014-08-12, 2027-01-16] [2026-01-18, 2013-03-24, 2024-01-17, 2024-02-18, 9999-12-31, 9999-12-31, 2023-12-19, 2024-08-03, 2024-02-18, 2025-06-18] [2024-01-09, 2024-06-30, 2024-01-09, 2023-12-17, 2023-12-15, 2023-12-19, 2023-12-13, 2026-02-18, 2024-01-09, 2024-01-09] [2023-12-16, 2025-06-18, 2023-12-10, 9999-12-31, 2023-12-18, 2024-02-18, 2026-02-18, 2024-01-17] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00] [2024-01-31 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [2023-12-10 00:00:00, 2023-12-13 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 2018-04-06 00:00:00, 2024-01-09 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00] [2023-12-09 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00] [2027-01-16 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000] [2024-08-03 13:08:30, 2023-12-14 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00, 2024-01-08 00:00:00] [2027-01-09 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00, 2025-02-18 00:00:00] [2023-12-14 00:00:00, 2024-01-19 00:00:00, 2023-01-15 08:32:59.123123, 2025-02-17 00:00:00, 2025-02-17 00:00:00] [2024-06-30 12:01:02.123000, 2026-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-17 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30] [2023-12-17 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00] +76 -1386695947 1577889961 -1545157062 -2147483648 2014-08-12 2025-02-17 2026-01-18 2023-12-20 库里环保局不好海星整合单位信号 get \N 就爱个令我进行 were 1 0 1.200000 40.123000 68.034300 0.000000 2023-12-09 00:00:00 2024-02-18 00:00:00 2023-12-12 00:00:00 9999-12-31 00:00:00 9999-12-31 23:59:59 9999-12-31 00:00:00 2023-12-12 00:00:00 2023-12-14 00:00:00 [0, 1, 1, 0, 0, 1, 1, 1, 0, 1] [1, 0, 1] [1, 0, 0, 0, 0, 0, 0, 0, 1, 0] [0, 1, 1, 0, 1] [127, 62, 0, 72, -128, 0, -1, 108] [127, 2, 1, 7, -7] [] [-128, -128, 24, -128, 2, 127, 1, 105] [-12515, 1, 32767, 245, 32767, -152, -1, 1, 245, 0] [32767, 245, 28049] [] [-5215, 8751, 4734, -21213, 28410, 32767, -1, -1, -16575, -3856] [-1, 987622064, -1, -382031033, 147483648, 32679, -895426800, 0, 32679, -2147483648] [0, 0, 2147483647, -2147483648, 0, 32679, 288621595, 880191409] [1, -782886162, -2094819934, -2147483648, -1459879460, 605271954, 1, -2147483648] [] [-2005949333, -1, 1] [9223372036854775807, -489571327883196071, 2624009556148233347] [3479305, 7662938207305941298, 0] [-109672033412292300, -1, 1425363396048035301] [-2, 2, -2, -2, 1, 0, 2, 1, 0, -1] [-1056695247, -2, -2, -2, -1547088992] [1064812701, 0, -1096767005, 0, -2, 8, -1, -717525493, 0, -27317157] [5, -1, 1, 907837753, -1, 0, -2, 0, 55567046, 4] [300.343000000000, 14.170100000000, 1.200000000000] [40.123000000000, 87.148400000000, 100.020000000000, 100.020000000000, 0.000000000000] [1.200000000000, 300.343000000000, 40.123000000000, 47.034900000000, 27.116900000000, 70.124400000000, 300.343000000000, 300.343000000000, 40.123000000000, 84.138500000000] [42.065800000000, 300.343000000000, 3.001300000000, -1.200000000000, 1.200000000000, 14.052000000000, 56.045800000000, 300.343000000000, 40.123000000000, 52.190600000000] [60.0669, 94.1097, 40.1230, 1.2000, 36.0145, 40.1230, 1.2000, 57.0452] [1.2000, 300.3430, 92.0114] [] [7.0778, -1.2000, 300.3430, 300.3430, -1.2000, 40.1230, -1.2000, 0.0000] [] [40, -1, 100, 11, 0, 40, 33, 300, 300, 40] [40, 35, 1, 0, -1] [1, 51, 87, 66, 0, 1, 52, 100] ["good had want with back", "so", "天晴", "look", "tell", "I'll she how", "be know some if did who got come will", "女孩", "标准", "is"] ["be could", "really", "-", "r", "l"] ["接收", "", "?", "中原路", "this like"] ["tell", "--", "黄委会"] ["-", "-", "爱情慧眼陇海人生", "才到", "g", "最新版本", "we", "me"] [] ["-", "?", "串货", "", "天威一首发布庄园群里理解", "m", "技术港湾半年都还", "去过平均天数机柜鼎力制动阀舍得心里的八一路去吃饭吧", "you're", "实实在在方法黄黎明五笔你好呀第二个一颗起来份款厦门"] ["取款都发生", "发邮箱核好应用考核什么的", "?", "相约晒后占用", "优于我们的副总着急把控知己", "with", "能不鞥对待新世纪", "捕鲸船动态前奏"] [] ["c", "a", "y", "w", "a", "m", "f", "f", "a", "v"] [] ["h", "d", "j", "x", "g", "v", "d", "n", "u", "z"] [2023-12-13, 2026-02-18, 2023-12-15, 2024-07-01, 2027-01-16] [2024-07-01, 2023-12-10, 2023-12-12, 9999-12-31, 2027-01-09] [2019-05-05, 2023-12-16, 2024-02-18, 2025-06-18, 2023-12-09, 2024-01-17, 2023-12-16, 2026-01-18] [2024-02-18, 2023-12-11, 9999-12-31] [2027-01-16 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-13 00:00:00, 2023-12-13 00:00:00, 2014-08-12 00:00:00] [2023-12-13 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00] [2025-06-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59] [] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2013-06-03 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00, 2023-12-16 00:00:00] [2024-07-01 00:00:00, 2024-07-01 00:00:00, 2026-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000] [9999-12-31 23:59:59, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2027-01-16 00:00:00] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2027-01-16 00:00:00, 2024-01-17 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00] [2014-08-15 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00] [] +78 2147483647 \N 1 -1487639878 2023-12-09 2023-12-09 2023-12-11 9999-12-31 华林煤气侯景祝你好吸纳中那款进入身份证对方相互互补 就在 - don't his okay him this back did they - go -- 1 0 49.017300 1.200000 100.020000 1.200000 2027-01-09 00:00:00 2023-12-10 00:00:00 2024-01-19 00:00:00 2023-12-17 00:00:00 2023-12-16 00:00:00 2023-12-12 00:00:00 2024-01-17 00:00:00 2023-12-12 00:00:00 [1, 0, 1, 1, 0, 1, 1, 1, 0, 0] [1, 1, 0, 1, 0] [0, 0, 0, 1, 1, 0, 0, 1] [1, 1, 0, 1, 0, 0, 0, 1] [70, -13, 7] [3, -1, 4, 0, 1] [] [-21, 0, 3, -1, -128, 1, 0, 2] \N [14351, 6095, -1613, 12262, 32767] [3419, 0, 0, -29069, 30445] [32767, -32768, -1667] [917258286, 365141111, -1893510758, 1, 1] [-1678957193, -1, 1] [-413471348, 1014766509, 688122117, -1131576772, -1647210168] [] [-9223372036854775808, 1, 3717606443996817700, 1, -9223372036854775808] [245, -4480947567765916307, -4608452321395992015, 245, 245] [32679, 1, 5159299, 2100914846, 32679] [32679, 147483648, 147483648, -1, 245] [1, 194623167, -1, 1, 9, 7, 1, 4] [4, -803686393, 7, 0, -2, 731652733, -1, -2, 1, -2] [1, 3, 1, -2, -1, -2, 0, -1] [2, 4, 1, 1, -1] [63.179900000000, -1.200000000000, 40.123000000000, 24.144900000000, 1.200000000000] [] [] [] [100.0200, 300.3430, 40.1230, 100.0200, 0.1701] [100.0200, 300.3430, 67.1574, 1.2000, 0.0000, 300.3430, 100.0200, 40.1230, 90.1587, 100.0200] [] [69.1628, 40.1230, 0.0000, -1.2000, 1.2000] [0, 22, 100, 40, 300, 20, 21, 300, 76, 36] \N [9, 40, 60] [46, 1, 1, 0, 76, 48, 300, -1, 300, 300] ["", "--", "?"] [] ["?", "壹仟伍博威幸福交通新鑫", "", "why do here i him if", "?", "看重", "快乐世家", "授权四天"] ["柏霖", "离开", "实现航务我们认识吗重要性零捌那是注明青青绿色架子", "最深刻列表满天飞一首有没有", "检测", "was got were in we", "参观", "then", "记得", "接近"] ["--", "?", "-", "热播方案没钱办款大棒熊猫大话", "were got a back from how that's so"] ["能早点三件", "good your don't look were time why", "?", "with of okay about mean how her", "f", "why got it's with want were not time of oh", "it's", "--"] ["?", "-", "", "丁丁不像可怕", "?", "--", "牡丹公司", "喝酒优于前辈云鹏", "本身冤枉是不是", "-"] [] ["i", "y", "w"] ["t", "c", "r", "m", "b", "s", "x", "r", "f", "r"] ["t", "f", "p"] ["d", "e", "w"] \N [9999-12-31, 2024-08-03, 2024-01-09, 2024-01-17, 2024-01-08] [2024-01-19, 2024-08-03, 2024-06-30, 2023-12-09, 2023-12-13] [2027-01-09, 2023-12-09, 2024-01-19, 2025-02-18, 2014-08-12] [] [2024-01-17 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00] [2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2003-02-17 00:00:00, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 9999-12-31 23:59:59] [] [9999-12-31 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00] [] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30] [2026-02-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00] [2023-12-11 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-19 00:00:00, 2026-01-18 00:00:00, 9999-12-31 00:00:00] [2023-12-15 00:00:00, 2025-02-17 00:00:00, 2001-08-10 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2005-11-21 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00] [2023-12-18 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00] +79 147483648 0 0 -333265026 9999-12-31 2023-12-15 2023-12-10 2023-12-16 他妈信天游来看资金大风合作一家这不是科嘉 心理学一点不够 no yes no it's one say - 权威 my go know have they a well me no 1 0 1.200000 70.194300 0.000000 44.122700 \N 2023-12-17 00:00:00 2024-01-08 00:00:00 2023-12-16 00:00:00 2023-12-17 00:00:00 2023-12-20 00:00:00 2023-12-15 00:00:00 2025-06-18 00:00:00 [1, 1, 1, 0, 1, 1, 0, 0, 0, 0] [1, 0, 0, 0, 1, 1, 1, 1] [1, 0, 0] [0, 1, 0, 0, 1, 1, 0, 1] [-1, 42, -1, 0, 0] \N [-19, -128, -128, -1, 2, 1, -128, -128, -8, 1] [93, 1, 0, -1, -100, -44, 1, 1, -45, 0] [20372, -1, -1, 243, 24021] [-1, -1, 32767] [1, -25675, 32767, 18598, -865, -9977, 6675, 1, -31006, 245] [-32768, -16531, -29805] \N [147483648, 0, -1, 1, 1933359281, 147483648, 1, 32679] [357112378, 0, 2147483647, -1, 151603727] [] [] [147483648, 245, -5280484051716123129, 147483648, 32679] [-2349567151549164433, -9223372036854775808, 0, 0, 32679, -1, 0, -4953496451440497323, 9223372036854775807, 1] [9223372036854775807, 9223372036854775807, 147483648, -7690936, 9223372036854775807, 8141457840035992836, 245, -1, 0, -526936690] [-1, -993252587, 1] [-1, 1227109777, -2, -1, 2, -1, 0, 8] [0, 1, 1] [-1, -2, -1506060057, -1, 0, -1, -2, 1] [0.000000000000, 49.137900000000, 40.123000000000, 32.114300000000, 34.099800000000, 76.018300000000, 41.024800000000, 40.123000000000, 100.020000000000, 7.005500000000] [-1.200000000000, -1.200000000000, 40.123000000000, 300.343000000000, 300.343000000000, -1.200000000000, 0.000000000000, 3.016500000000] [] [300.343000000000, 40.123000000000, 0.000000000000, -1.200000000000, -1.200000000000, 90.091500000000, -1.200000000000, 34.016100000000] [40.1230, 40.1230, 40.1230, 100.0200, -1.2000] [300.3430, 1.2000, 0.0000] [17.1799, 1.2000, 52.1395, 40.1230, 0.0000, 42.1741, 0.0000, -1.2000, 100.0200, 0.0000] [90.1961, 47.0416, 300.3430, 300.3430, -1.2000, 300.3430, 87.0337, 60.0116, 75.0858, 56.1689] [-1, 0, 300] [100, 100, 1, 300, 0] [100, 77, 300, 66, 61, 81, 90, 95] [-1, -1, 77, 0, 1, 51, 0, 300] ["with", "", "系统集成客运一件短语"] ["when", "彩霞无论相关员工", "for", "", "-", "胡华威", "他人", "有波验证名单深蓝"] ["my", "--", "yes this like at a you", "序列号也有过找到查证我在同学家没有总经销商", "going", "格飞", "分别对咱们刘畅相识厂家熟悉可怕说不定", "can't", "think", "自导交流"] [] ["大棒", "", "--", "--", "b", "孙健", "mean", "放弃", "some his me that's for ok a", "?"] ["but", "海尔周期故事你还不去下个过去价格前辈河南电子卡将", "显卡补丁呵呵娃娃陪着双千兆平顶山", "didn't how", "I'm", "那种集群我就造成分数这种明显", "汇众做生意婚嫁", "分开", "y", "?"] ["--", "-", "盛源"] ["原厂", "鸿锐", "oh", "--", "okay all how because were now hey can't could", "?", "--", "-"] ["x", "e", "c", "o", "s", "l", "u", "h", "q", "z"] ["p", "i", "n", "b", "z", "y", "f", "q"] ["n", "u", "r", "k", "x"] [] [2026-02-18, 2027-01-16, 2023-12-12, 9999-12-31, 2024-07-01] [2026-01-18, 2024-01-09, 2023-12-16, 2024-02-18, 2023-12-09] [2024-02-18, 2024-06-30, 2023-12-11] [2023-12-17, 2024-01-09, 9999-12-31, 2023-12-12, 2023-12-09] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00] [] [2024-01-19 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59, 2023-12-09 00:00:00, 9999-12-31 23:59:59] [2024-02-18 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00] [2025-02-17 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00, 2024-01-19 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-12-13 00:00:00] [2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00] [2024-06-30 12:01:02.123000, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59] [2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00] \N [2023-12-20 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00] [2025-02-17 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59.123123, 2027-01-16 00:00:00, 2025-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00] [2024-01-19 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] +81 -1 1692736526 590406635 1381103336 2000-10-24 2027-01-09 2026-02-18 2024-06-30 -- 纯平那么泥石谁人家云隐 \N time got he's something got good look and good is with \N 0 1.200000 300.343000 1.200000 0.000000 2024-01-09 00:00:00 2026-01-18 00:00:00 2024-01-17 00:00:00 2024-01-17 00:00:00 9999-12-31 23:59:59 2023-01-15 08:32:59.123123 2023-12-15 00:00:00 2024-01-08 00:00:00 [0, 0, 0, 0, 0, 0, 1, 1, 1, 1] \N [0, 0, 1, 0, 1] [] [0, 14, 39] [72, -128, -1, -73, -103] [-128, 1, 0, -128, 1] [65, -128, 9, -83, 127] [-1, 32767, -10271, -32768, 29040] [1091, -11338, 4764, 32767, 0] [1, 0, -1] [0, 0, 1721, 28841, 27375, -10626, 32767, 18711, 245, 23211] [2147483647, -1777716039, 2147483647, 1988438794, 2147483647] [32679, 1, 1, 1, 0] [] [820268243, 0, 2147483647, 1, 2147483647] [] [-1, -9223372036854775808, 9223372036854775807, -9223372036854775808, 9223372036854775807] [-1, 6690351, 0, 147483648, 1612790987933988505, 4907429395391335252, 2738675386156863961, 8152804963810518224] [1, 5598971, -5118713753182895279, -1, 245, 245, -7794587505905642771, 147483648] [-1, 1, -2, -2, 0, 384530761, 1, 8] [] [] [0, 1, 0, 0, 4, 0, 0, -2] [300.343000000000, -1.200000000000, 26.029500000000, 0.000000000000, 100.020000000000, 300.343000000000, -1.200000000000, 40.123000000000] [0.000000000000, 37.072100000000, 1.200000000000] [100.020000000000, 47.144600000000, 40.123000000000, 0.000000000000, 86.091100000000, 1.200000000000, 40.123000000000, 300.343000000000, 28.059800000000, 55.086100000000] [-1.200000000000, 5.172100000000, 1.200000000000, -1.200000000000, -1.200000000000, 89.181700000000, 1.200000000000, 18.198700000000, 100.020000000000, 300.343000000000] [1.2000, 300.3430, -1.2000, 300.3430, 94.0788, 300.3430, -1.2000, 8.0639] [40.1230, 14.0988, -1.2000, 100.0200, 71.1036] [1.2000, 4.1994, 1.2000, 100.0200, -1.2000, -1.2000, 79.1924, 300.3430] [61.0206, 40.1230, 36.1116, 300.3430, 74.1488, 59.0967, 100.0200, -1.2000, 64.0808, 100.0200] [40, 1, 7, 300, 98, 300, -1, 41, 300, 0] [300, 300, 12] [54, 100, 0] [0, 54, -1] ["", "联系过", "going", "ok", "s", "-", "and it's got or she don't she come", "昨晚上", "", "顺驰环境"] \N ["me mean at my can me right the no from", "曙光项目汉威纯平农行一天经三路对了", "代玉", "下半年市委无所谓一群次要空分海川洗液法律", "s"] ["?", "多些", "?", "-", "--", "赔十", "", "渠道我也在联系人真正以以上的全程红颜", "that's because can't then here but her a know", "--"] ["?", "--", ""] ["有你们做过世家", "产生意义产生蓝天湖北龙飞两台万能早点", "right be at well the would", "我们的泰杰斯", "say", "-", "tell", "as as they about tell in for", "were I'll all as", "k"] ["另外发到淀雪华林制作上面", "-", "something got it's", "it's my", "--"] ["not", "-", "or going had", "get that for him that's to you're I'll who say", "-", "?", "指定机关这个心里的焦煤多少预算不完了吗", "when", "can't my so how if about with because", "?"] ["b", "l", "o", "c", "w"] \N [] ["g", "f", "s"] [2023-12-10, 2023-12-09, 2027-01-09, 2027-01-09, 2014-08-12, 2025-02-17, 2023-12-14, 2023-12-19, 2027-01-16, 2023-12-13] [2023-12-10, 2023-12-14, 2024-01-31] [2027-01-16, 2025-06-18, 2024-08-03] [] [2014-08-12 00:00:00, 2006-11-02 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00] [2023-01-15 08:32:59, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2023-12-12 00:00:00] [9999-12-31 23:59:59, 2024-08-03 13:08:30, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2024-01-19 00:00:00] [2024-06-30 12:01:02, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00] [2014-08-12 00:00:00, 2023-12-14 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2023-12-09 00:00:00] \N [2025-06-18 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00] [] [2024-06-30 12:01:02.123000, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2024-01-19 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-09 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00] [2023-12-13 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00] [2023-12-19 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00] +82 -2147483648 -585119102 -2147483648 -734263406 2023-12-10 2025-06-18 2026-01-18 2025-02-17 有过 ? who why \N that ? really there it's up why we think my what as really \N 1 \N \N 300.343000 0.000000 2026-01-18 00:00:00 2024-01-08 00:00:00 2023-12-17 00:00:00 2025-02-18 00:00:00 2023-01-15 08:32:59.123123 2025-06-18 00:00:00 2026-01-18 00:00:00 2024-06-30 12:01:02.123000 \N [0, 1, 1, 0, 1, 1, 0, 0, 0, 1] [0, 1, 0] [1, 0, 1, 1, 0, 0, 0, 0] [] [0, -128, 1, -1, 61, 1, 0, 0, 1, 1] [-128, -1, 22, 0, 0, -42, 127, 1] [] [0, -29639, 26303, 0, 32767] [24374, 1, -28600, 17257, 0] [] [] [0, 32679, 971307242, 1097102896, 32679] \N [95058356, -2147483648, 147483648, -410028821, 0, -490944477, 0, 0] [-2147483648, 32679, -902862799] [32679, -415235136625309385, 32679, 32679, -7414334428706968969, 838979663, -1844561065, 147483648, 32679, 9223372036854775807] [32679, 0, 2906969668045616140, 1455205596, 245, 147483648, -1, 7002071187100645500, 0, 32679] [-6774544, -139849, -8907326954883253449, 1, -1, 32679, 655360480, 6015245378149712901, 0, -9223372036854775808] [245, 0, 0, -1, -1, 9223372036854775807, -3506988147566968828, -1] [2, 0, 0, 7, -2] [-1, 1, 1, 0, 1, 4, -571106309, 1, 1113714371, 0] [0, 5, 598438349, 1, -2, 5, 5, 1, 1, 0] [] [1.200000000000, -1.200000000000, 10.110000000000, 0.000000000000, 54.174200000000] [31.132500000000, 40.123000000000, 88.188200000000, 0.000000000000, 96.044900000000, 24.024500000000, 76.074500000000, 40.123000000000] [40.123000000000, 40.123000000000, 40.123000000000, 39.012300000000, 40.123000000000, 47.023000000000, 100.020000000000, 300.343000000000, -1.200000000000, 300.343000000000] [1.200000000000, 300.343000000000, 66.155400000000, 1.200000000000, 1.200000000000, 100.020000000000, 40.123000000000, 1.200000000000] [13.1087, 40.1230, 100.0200, -1.2000, 40.1230, 45.0381, 51.1265, 0.0000, 65.0052, 0.0000] [0.0000, 100.0200, 300.3430, 40.1230, -1.2000] [31.1236, 100.0200, -1.2000] [] [87, 0, 100, 40, 45, 40, 55, 100, 0, -1] [15, 100, 1, 1, -1] [] [1, 0, 80, 78, 300, 0, 40, 40, 300, 93] ["留言飞机新宇", "--", "one didn't were for when"] [] ["大山海涛实现航务", "-", "稳定", "看不懂", "can he's hey not have in that", "新联惠机型欧典早晨刘广新的", "--", ""] ["", "and hey ok well she like", ""] ["be or", "-", "did it come yes because with did have like here"] ["go", "好不好算数绿城此外正信小计就不单身就像远方", "?", "下个联盛高密度图形工作站中联系", "责任"] ["?", "单单", "考前", "like good one when oh no could", "保卡"] ["-", "k", "真是天空安排一周华康利盟信息部汇众谈下来亿欧元随时"] ["a", "x", "q"] ["d", "t", "u", "w", "k", "f", "a", "j", "e", "s"] ["s", "v", "n", "r", "l", "s", "t", "g", "n", "s"] ["g", "j", "y", "o", "s", "k", "l", "z"] [2023-12-13, 2023-12-16, 2023-12-18, 2023-12-15, 2027-01-09, 2023-12-17, 2025-06-18, 2017-12-10] [] [2024-06-30, 2024-01-09, 2023-12-20, 2023-12-17, 2024-01-08, 9999-12-31, 2025-02-18, 2023-12-14, 2026-01-18, 2025-06-18] [] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2010-09-10 00:00:00, 2003-09-05 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00] [9999-12-31 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2024-01-19 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00] [2024-01-09 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00] [] [2025-06-18 00:00:00, 2026-01-18 00:00:00, 2023-12-17 00:00:00] [2027-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59.123000, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30] [2027-01-09 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-11 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00] [2023-12-15 00:00:00, 2026-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00] \N [] [2025-06-18 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00] +83 1801133090 2147483647 1 365385895 2024-08-03 2023-12-10 2025-06-18 2024-02-18 see -- want ? how o ? would \N 1 69.056800 300.343000 6.091200 300.343000 2023-12-14 00:00:00 2025-06-18 00:00:00 9999-12-31 00:00:00 2024-02-18 00:00:00 2027-01-09 00:00:00 2023-12-09 00:00:00 2024-06-30 12:01:02.123000 2026-01-18 00:00:00 [1, 1, 1, 0, 0, 1, 1, 1] [] [0, 1, 0, 1, 0, 0, 1, 0] [1, 1, 0, 1, 1, 1, 1, 1] \N \N [-128, -91, 1, 3, -1, 127, 1, -128, 1, 1] [56, 118, -128, 0, 7] [31281, 245, 5945] [0, -11042, 6835, 14511, 10346, 245, -561, -21834] [] [-1, 245, -29447, -32768, -25206] [2147483647, 1, -2147483648, -521800815, 2147483647, 147483648, 32679, 1412848738] [1, 147483648, 1] [-2147483648, -358178357, -533669781, 1, 0] [-801986853, 2147483647, 119511603, -1410865916, 1674764765, 147483648, -576250380, -195149046] [-2670092084673219837, -9223372036854775808, 2958150612331493383] \N [9223372036854775807, 32679, 7955823433824789936] [-1, 147483648, -1413321449, -3960110856727898303, 147483648, 147483648, -9223372036854775808, 652731240598003143] [-1, -2, 0] [1, 1, 6] [9, 0, -1, 0, 1, 66803470, 4, 749999832] [-1, -1, -1, 0, 1, 646361958, -1, -2] [40.123000000000, 100.020000000000, 1.200000000000, 0.000000000000, 300.343000000000, 40.123000000000, 100.020000000000, -1.200000000000] [] [79.116500000000, 98.113600000000, 18.034200000000, 15.115800000000, 31.153400000000, 0.000000000000, 35.169900000000, 40.123000000000] [40.109000000000, 40.123000000000, 0.000000000000, 300.343000000000, 1.200000000000, -1.200000000000, -1.200000000000, 100.020000000000, 40.123000000000, 5.144300000000] [40.1230, 69.1158, -1.2000] [1.2000, 98.0627, 0.0000, 94.0017, 40.1230] [96.0993, 30.0047, -1.2000, -1.2000, -1.2000, 40.1230, 0.0000, 20.1907] [] [-1, -1, 83, 0, 300, 62, -1, 56] [8, 89, 40, 300, 40, 0, 0, 40, -1, 1] [40, 300, 100] [40, 40, 58, 1, 300, 40, 300, 300] ["那么", "back", "", "国企不久联系人", "?", "yeah she good would that's I'll me", "in you're time up that's you're back that of", "time"] ["三阳我打印出来个给你吧凯峰高高兴兴青青绿色水货回去主要", "位居", "", "had now ok I'll from come yes it on", "it's"] [] ["this who so all the because how", "I'll", "热线鑫创爱国沉住气", "from have", "f", "?", "-", "a as out a"] ["打工的", "", "-"] ["--", "先这样昨天到达众诚慧眼深爱大部分单单我刚来呀张先波发货", "back"] ["不到位中华", "不对战友", "中心", "除外", "I'll now her with go with i time like you're", "who", "?", "心情"] ["有限文宁在线做事完成长春市前两天购买", "out been got well didn't she", "want just you have he like why or when", "-", "看不出来", "but", "or my", "你呢岂能后天回五千", "some", "王全得"] ["l", "p", "f", "l", "c", "g", "e", "i"] \N ["m", "t", "u"] ["c", "x", "o", "u", "v", "d", "j", "q"] [2023-12-12, 2024-02-18, 2024-01-17, 2023-12-19, 2023-12-11, 2023-12-12, 2023-12-17, 2026-01-18] [] [2027-01-09, 2025-02-18, 2024-01-09, 2023-12-12, 2023-12-16, 2025-02-17, 2023-12-17, 2016-11-24, 9999-12-31, 2024-01-31] [2023-12-18, 9999-12-31, 2023-12-16] [2023-12-09 00:00:00, 2027-01-09 00:00:00, 2000-06-25 00:00:00, 2024-01-31 00:00:00, 2026-02-18 00:00:00] [9999-12-31 23:59:59, 2026-01-18 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2027-01-16 00:00:00] [] [] \N [2023-12-20 00:00:00, 2024-01-08 00:00:00, 2008-02-01 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [] [2025-02-17 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00, 2024-01-09 00:00:00] [2023-12-10 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00, 2001-04-12 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00] [] [] [2024-07-01 00:00:00, 2013-11-27 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00, 2023-12-14 00:00:00] +84 -2147483648 1009668588 147483648 -2147483648 2024-01-17 2024-01-09 2025-02-18 2025-06-18 c what \N my 奖励 -- his 0 1 -1.200000 15.189300 40.123000 -1.200000 \N 2024-02-18 00:00:00 2006-08-12 00:00:00 2024-02-18 00:00:00 2026-02-18 00:00:00 2024-02-18 00:00:00 2023-12-14 00:00:00 2025-06-18 00:00:00 [1, 1, 0, 0, 0, 1, 0, 1, 1, 0] [0, 1, 0, 0, 0] [1, 1, 0] [1, 0, 0, 1, 0, 0, 1, 1] [17, -1, 6] [-128, -128, -128, -63, 6, 1, 1, -21, -65, -1] [-1, 71, 127, 1, -1, -128, 1, 119, -1, -111] [81, 0, 66, 0, 1] [-1, -2967, 245, -32341, -10247] \N [] [-1, 32767, 32767] [655200070, 2147483647, -1591113680] [0, -1666721546, -1, 147483648, -1, -1, -2147483648, -1240095645] [] [-1671586508, 147483648, 0, 878931689, 921441554, 147483648, 2033877275, 0] [8207055908110953854, 1, 32679, 147483648, -5709782771854250832, 4566321264057774017, -1, 1, 32679, -4170445050348156911] [3482110949807048966, 3396072675664438183, 2091921423540721955, 8765430130744291491, -9223372036854775808, -9223372036854775808, 147483648, 32679, -3692688, -9223372036854775808] [3968070916537283666, -7782315930088332528, 32679, 0, -1, 9223372036854775807, 147483648, -7632402646853047149, -5127634252245739021, -1] [4694039155919540198, 4180018573846399110, 32679, 9223372036854775807, 1, 0, 4128139968219545737, 32679] [1, 1, 4, -470205362, 0, 1, -2, -1] [-403745002, 2, -2, 0, -2] [0, -1, -2] [5, -1, 8] [-1.200000000000, 1.200000000000, -1.200000000000, 21.044700000000, 0.000000000000, 64.162600000000, 0.000000000000, 40.123000000000, 100.020000000000, 300.343000000000] \N [-1.200000000000, 71.058700000000, 0.000000000000, 40.123000000000, 0.000000000000, 26.195000000000, 0.000000000000, 78.110100000000, 97.184500000000, 100.020000000000] [18.068800000000, 40.123000000000, 100.020000000000, 45.086300000000, 40.123000000000] [51.0497, 31.0053, 40.1230, 1.2000, 40.1230] [40.1230, 74.0095, 300.3430] [] [70.0296, 40.1230, 100.0200, 50.0729, 1.2000, 94.1136, 40.1230, 300.3430] \N [300, 300, 56, 0, 300, 300, 87, 1] [63, 93, -1, 1, -1, -1, 8, 42] [98, 300, 300] ["or", "x", "订货", "那次丁老师驱动器幸福线缆杨学规模", "-", "a", "get", "--", "联系一览表", "from want"] [] ["it's", "初八因为过程", "to", "欣慰", "?"] [] [] ["--", "硬件", "did"] [] ["推荐乔经理不分汉字", "I'll", "", "?", "", "好心", "not", "there his had why that's like", "珍贵九九李景林有什么人员中小型摩擦不论", "good"] ["f", "y", "p", "x", "t", "z", "n", "v"] ["v", "d", "h", "e", "f", "b", "q", "w", "q", "f"] ["w", "q", "i", "t", "f", "r", "g", "n"] [] [2024-07-01, 2025-06-18, 2014-08-12] [2024-01-31, 2023-12-09, 2024-07-01] [2027-01-09, 2024-08-03, 2024-01-31, 2010-03-21, 9999-12-31, 2023-12-16, 2024-08-03, 2025-06-18, 2025-06-18, 2023-12-20] [] [2023-01-15 08:32:59, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2024-08-03 13:08:30, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-07-01 00:00:00] [2027-01-16 00:00:00, 2025-02-18 00:00:00, 2003-03-21 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00] [] [] \N [2023-12-11 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00] [2025-02-18 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000] [2024-01-19 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00] [9999-12-31 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00] [2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [2027-01-16 00:00:00, 2023-12-16 00:00:00, 2023-12-15 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00] +85 -1 -611156656 1427104175 1 2024-01-17 2026-01-18 2023-12-14 2024-02-18 - say just now did it's yeah when come ? q ok see will like there can what if some and oh she going ? 1 1 94.113300 19.075800 300.343000 40.123000 2025-02-18 00:00:00 \N 2023-12-11 00:00:00 9999-12-31 23:59:59 2025-02-18 00:00:00 9999-12-31 23:59:59 2027-01-16 00:00:00 2023-01-15 08:32:59.123123 [1, 1, 0, 0, 1, 1, 1, 1] [0, 1, 0, 0, 1] [0, 1, 0, 1, 1] [1, 1, 0] [0, -111, 0, -1, -6] [-128, 8, 72] [] [0, 19, -123, -128, -17, 127, 77, 127, 127, -109] [-23831, -21813, 20056, 32767, -1] \N [-24533, 32767, -23575, 2005, 3724, -22903, 1, -1, 0, 5017] [-7307, 245, -32768, -1, 32767, -3953, -25705, -31271] [0, 2147483647, 1349225138, 1254066545, 1968789106, -1, 1860488905, 147483648] [] [1767280636, -1246471611, -154627238, 1216061323, 647046006, 2147483647, 32679, 1134065950, -716372413, -1096624105] [-1782604904, 2147483647, 0, 821699008, 1502819037, -1519821828, 560963917, -2147483648, 2147483647, 1] [-1, 9223372036854775807, 0, 1, -1380165] [] [0, 6920033909749472669, 147483648, 245, 245] [] \N [0, 0, -1077090410, 6, 8, 1083772083, -1, 0, -1627872036, 8] [-1, -2, 1, -2, 1, 3, 4, -2] [0, 1772360840, 1] [-1.200000000000, -1.200000000000, 19.163600000000] [1.200000000000, 93.005100000000, 60.035400000000, 300.343000000000, 1.200000000000] [40.123000000000, 53.090000000000, 24.129400000000, 34.194400000000, 68.134100000000, 100.020000000000, 28.063300000000, 39.065100000000] [300.343000000000, 2.088700000000, 1.200000000000, 85.066200000000, 100.020000000000, -1.200000000000, 32.199900000000, 0.000000000000] [56.1934, 45.1794, 100.0200, -1.2000, 46.1023, -1.2000, 10.0214, 0.0000] [] [1.2000, 1.2000, 65.0949, 66.0086, 100.0200, 40.1230, 300.3430, 40.1230] [] [300, 45, 4, 100, 100, 300, 25, 300] [300, 100, 51, 1, 300, 0, 0, 1, -1, 21] [1, -1, 0] [55, 40, 0, -1, 61, 0, 1, 0] ["x", "something they there something", "on can they of", "鸡巴", "think well had why go come"] [] ["who how ok not who", "唱给", "--"] ["go can here will one were think", "there", "壹仟伍", "-", "did"] ["on", "?", "g", "对于我无论", "-", "", "一米内部问了占有拥有引进过来刘畅顺驰", "", "I'm the or in not my oh", "-"] ["i not to going go then that's he my you're", "--", "伤心研普静静动物不够信源", "how don't", "about"] ["元旦联恒这一块图形工作站", "?", "--", "--", "结果不给等于天地一万天慧", "were", "", ""] ["b", "they a what is now got", "期间", "难过", "-", "--", "刻骨铭心", "p", "x", "think been how look going one her so get did"] ["n", "p", "m"] ["i", "f", "a", "p", "v"] ["l", "o", "x"] ["y", "q", "t", "i", "d"] [] [2023-12-18, 2025-06-18, 2024-01-09] [] [] [2026-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00] [2023-12-13 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00] [] [2024-01-09 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00] [] [2024-02-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-06-30 12:01:02.123000] [2024-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-19 00:00:00, 9999-12-31 23:59:59] [] [2024-01-19 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00] [2015-08-26 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123123, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00] +86 938867303 1 32679 671188025 2025-06-18 2024-08-03 2023-12-18 2025-02-18 can - 哟偶那个儿时这一说 u have back your had right 北站 1 0 0.000000 300.343000 68.164400 100.020000 2023-12-17 00:00:00 2025-02-17 00:00:00 2018-10-23 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2023-12-18 00:00:00 2025-06-18 00:00:00 2024-07-01 00:00:00 [0, 1, 0] [0, 1, 1] [1, 1, 1, 0, 1, 0, 0, 0] [1, 1, 0, 0, 1, 1, 0, 0, 0, 1] [1, 0, -1] [-128, 127, 1, 83, -103, 127, -120, 106] [-118, -37, -128, 7, -102] [-128, 2, 1] [-167, 82, 24985, 3346, 0, -11932, 1, 18743] [1, 1, 31522, -32768, 10731, -10429, -6795, 8158, -32768, 284] [30702, 32767, 30062, 26241, 1, 30369, -15130, -7857, 1, 27284] [-1, 245, -32768, 0, -17550] [2147483647, -2147483648, 1] [2147483647, 0, 147483648, 2147483647, 2147483647, 1, 547751934, 601651747] [] [-2132223004, 1, -1747251923, 1551201038, 2147483647, 1, 1122395912, -78218761, 60925632, 2067244169] \N [-7512260, 300452522, -808575227, 32679, 8638917660495007618] [32679, 147483648, 0] [32679, 245, 147483648, 245, 7588534135341964799] [-1460455666, -1, -1, 1, 1639118882, -1166286697, 0, -2, 1, 0] [1805335905, 1, 0, -1586088750, -1] [-2, -1, 0, 0, 1] [] [100.020000000000, -1.200000000000, 100.020000000000, 1.200000000000, 300.343000000000, 79.171000000000, 300.343000000000, 91.100100000000] [-1.200000000000, 1.200000000000, -1.200000000000, 33.003600000000, 10.134200000000, 1.200000000000, 0.000000000000, 40.123000000000] [13.032000000000, 300.343000000000, 56.120100000000] [] [300.3430, -1.2000, 300.3430, 93.0253, 35.1085, 1.2000, -1.2000, 1.2000, 54.1550, 40.1230] [-1.2000, 55.1594, 300.3430] [0.0000, 0.0000, 300.3430, 71.0052, -1.2000, 300.3430, -1.2000, 0.0000, -1.2000, -1.2000] [100.0200, 300.3430, 40.1230] [1, 19, 300, 1, 93, 1, 12, 1, 40, 82] [-1, -1, 300, 42, 40] [17, -1, 62, 300, 40, 100, 300, 1, 69, 0] [94, 100, 0] ["would", "动物改动易插拔天数饲料许昌怎么办能用吗还有李平", "--"] ["a", "-", "", "公寓", "下去", "简历郑大随风跃民", "", "don't", "see", "here"] [] ["", "z", "that okay when been up", "摄像头", "--", "because that's it's just yes when so it I'm get", "?", "with is it's i", "经济", "something not see can right see got a"] [] ["p", "--", "have the was be with", "-", "x", "学习", "特配", "c"] ["?", "about", "are I'll back look of it get that do"] ["okay", "-", "x"] \N ["e", "t", "h", "a", "p"] ["h", "a", "d"] ["g", "x", "v", "z", "u", "e", "j", "r"] [2024-08-03, 2017-04-23, 2023-12-09, 2024-02-18, 2025-02-17, 2027-01-09, 2023-12-10, 2023-12-17, 2024-01-31, 2024-02-18] [2014-08-12, 2023-12-09, 2023-12-14] [2024-01-08, 2014-08-12, 2023-12-10, 2026-01-18, 2023-12-19, 2024-02-18, 2024-02-18, 2025-06-18] [2026-01-18, 2023-12-16, 2000-10-11, 2023-12-14, 2024-01-08, 2023-01-15, 2024-01-08, 9999-12-31, 2024-01-17, 2023-12-15] [2025-02-17 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00] [2025-06-18 00:00:00, 2023-01-15 08:32:59, 2023-12-09 00:00:00, 2023-01-15 08:32:59, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [2024-01-08 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00] [2023-12-09 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00] [2024-01-31 00:00:00, 2024-01-17 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] \N [] [2023-12-10 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00] [2023-01-15 08:32:59.123123, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2024-01-19 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [] [2025-06-18 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00] +87 1124126096 228086539 1011725045 147483648 2025-06-18 2024-06-30 2000-07-17 2007-09-28 又将 - -- -- ? 批发价大量一款手里这么周岁含税微波 0 0 0.000000 35.007300 55.108100 57.104000 2025-06-18 00:00:00 2024-01-09 00:00:00 2025-06-18 00:00:00 2027-01-16 00:00:00 2023-12-17 00:00:00 2024-01-19 00:00:00 2023-12-18 00:00:00 2024-01-09 00:00:00 [1, 0, 1] [0, 1, 1, 0, 1, 1, 1, 0] [1, 0, 1] [1, 1, 1, 1, 1] [1, -1, -128] [84, 5, -91, -128, 127, 1, -128, 0] [5, -1, -1, 0, 127] [-1, 22, 0, -1, 0] [] [-16973, 12869, -27714, 1, 245, 13663, 24169, -5471, -25965, -697] [-4953, 32101, -31499] [-1, 22525, 22857] [118176411, -1, 361301827, -1, 147483648] [1801062124, -863029636, 147483648, -2147483648, 147483648, 32679, 1593722044, -877258564] [794753912, 1531786284, 2147483647, 1840507934, -247796117] [1525492400, 32679, 2147483647, 1167530877, 2147483647] \N [-9223372036854775808, -1181807280989426709, 245, 245, -9223372036854775808] [-1002481, 9223372036854775807, 32679] [1323095806, 147483648, 245, 32679, 147483648, 0, 245, 1300811905, -1, -1499561884] [-2, 0, 4, 64210075, 8, -2, 612878217, -2, 0, 1] [0, -2, -2] [-2, -106724829, 1389264911, 6, 9, -2, 1293172431, 0, -8911486, 0] [] [40.123000000000, 40.123000000000, 100.020000000000] [34.067600000000, 0.000000000000, -1.200000000000, 4.106400000000, 65.150100000000, -1.200000000000, 0.000000000000, 1.200000000000, 19.038300000000, 1.200000000000] [0.169200000000, 100.020000000000, 40.123000000000, 300.343000000000, 1.200000000000] [1.200000000000, -1.200000000000, 1.200000000000, 34.026800000000, 300.343000000000, 8.148300000000, 300.343000000000, 40.123000000000, 100.020000000000, 300.343000000000] [61.0914, 40.1230, 1.2000, 1.2000, 95.1571, 40.1230, 100.0200, 75.0600, 49.1840, 100.0200] [40.1230, 50.0053, 83.0731, 1.2000, 49.0148, 40.1230, 13.0979, 47.1822, 15.0293, 0.0000] [56.0153, 25.0578, 40.1230, 11.0857, 64.1650, 100.0200, 56.0109, 51.0714, 40.1230, 8.0942] [91.0626, 300.3430, 71.1607, 47.1916, 71.0624, 40.1230, -1.2000, 81.1555, -1.2000, 40.1230] [] [4, 91, 27, -1, 40] [100, 46, 45] [-1, 92, 0, 80, 100, 87, 64, 81, -1, 100] ["?", "", "总价", "提供几十年代理商讲师农历咨询影视", "连成信号看不懂"] [] ["成功文档", "--", "his", "类似见过东海拜拜对方显示保险编号元旦", "?", "see", "--", "艾蕊", "", "驱动华阳星月早点肯定连接线主做"] ["远方天威天气壹佰", "三门峡第二款回来电话给我西郊", "第二次网页", "智者远方红色博思系统集成商", "r", "--", "", "yes", "would didn't me is mean all yes", "弱阳性"] ["可以到不久赶快代表越来越来", "--", "going been time this see", "王青", "?"] ["质保我不会玩协作也有过伏牛路一级也要", "处理事情宣传播放为难小机器尤其出差签单谁家", "--", "-", "could"] ["it", "放心不欢迎梁英如何", "-"] ["经营", "--", "我不会玩", "or really ok can for what could", "it's yes him yeah him to can yes there", "", "there", "放心"] ["o", "k", "c", "i", "j", "j", "l", "a"] ["j", "v", "a", "i", "i", "d", "i", "c"] ["w", "d", "f", "b", "l", "r", "q", "o"] ["t", "i", "v", "r", "y", "j", "g", "d", "j", "d"] [2024-02-18, 2024-02-18, 2014-04-08, 2023-12-19, 2023-12-14] [2023-12-15, 2023-12-16, 2023-01-15, 2023-12-17, 2024-02-18, 2024-02-18, 2024-02-18, 9999-12-31, 2025-06-18, 2024-06-30] [9999-12-31, 2023-12-17, 2023-12-16, 2025-06-18, 2023-12-20] [2024-01-19, 2027-01-09, 2024-08-03, 9999-12-31, 2026-02-18, 2027-01-09, 2023-12-12, 2023-12-11, 2023-12-09, 2023-12-12] \N [2023-12-14 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2023-12-10 00:00:00] [2024-07-01 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00] [2024-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00] [9999-12-31 23:59:59, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00] [2014-08-12 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00] [9999-12-31 23:59:59, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00] [] [2027-01-16 00:00:00, 2024-01-09 00:00:00, 2025-02-18 00:00:00, 2023-12-12 00:00:00, 2024-01-31 00:00:00] [2019-09-15 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00, 2024-01-19 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123] [2008-01-14 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00] [2023-12-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2027-01-09 00:00:00, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00] +89 -1231389290 -1890596681 -1388457677 -1 2024-02-18 2023-12-11 2024-02-18 2024-08-03 can 保护来处于齐村还向知己加密金总我们认识吗费用青青绿色 -- your - ? 牡丹旅之星富强部分官方怎么 x 1 0 59.188000 -1.200000 100.020000 40.112300 2023-01-15 08:32:59.123000 2025-02-18 00:00:00 2024-02-18 00:00:00 2024-08-03 13:08:30 2024-02-18 00:00:00 2024-01-17 00:00:00 2025-02-18 00:00:00 2023-12-11 00:00:00 [1, 0, 1, 0, 1, 0, 0, 1] [0, 0, 0, 0, 0, 1, 0, 1] [0, 1, 0, 1, 0, 0, 0, 1] [1, 1, 0, 1, 1, 1, 1, 1] [0, 1, 127, -1, 1] [] [-128, 1, 1, 1, 60, 1, -112, -128] [2, -1, -106, -128, -128] [-1, -1, 15380] [27104, 22867, 31306, -5657, 4095, 0, -25543, 1, -32768, 127] [32767, -1, 26073] [-31516, 1, 14445, -28267, -16078, 25998, -1, 32767, -1, 25957] [2094714231, 1, 1] [32679, 147483648, 1894538880, -1996154246, 147483648, -516998498, -855409944, 0, 528349767, 156417406] [-2078045002, 2142670350, -1] [400849639, 1529073840, -1187462150, 147483648, 147483648, 1330798426, 1, -2044452902] [9223372036854775807, -588645660, 7720564127188028491, 1467063930, -1] [-2401091244258223090, 245, 5301256313780925317, 32679, -9078638221700983013] [0, 4135432, -153348, 245, 9223372036854775807, 0, -1, 1, 210608077, 9223372036854775807] [] [1, 0, 0, 1030247702, 1922029006] [-1, 4, 5, 1, 0, -1, 1, -236461767, -2, 9] [1, 1, 1344271914, 1, 1, -1, -2, 0] [1, 1, -1] [1.200000000000, 31.089000000000, 40.123000000000, 93.088100000000, 300.343000000000] [-1.200000000000, 65.155700000000, 0.000000000000] [1.200000000000, 300.343000000000, 85.142800000000, -1.200000000000, 300.343000000000, 40.123000000000, 300.343000000000, 63.014300000000, 0.000000000000, 72.157000000000] [0.000000000000, 22.166400000000, -1.200000000000, 100.020000000000, 15.177100000000, 76.013400000000, 40.123000000000, 91.159100000000] [11.1653, 84.0022, 0.0000, 1.2000, 0.0000, -1.2000, 40.1230, 100.0200] [300.3430, 1.2000, 300.3430, 99.1902, 15.0174, 100.0200, 27.1935, -1.2000] [0.0000, 300.3430, 300.3430, 1.2000, 40.1230, -1.2000, 57.1872, 300.3430, 300.3430, 40.1230] [-1.2000, 62.1052, 41.1015, 40.1230, 40.1230] [1, 100, 70, 100, 0] [100, 40, 100, 13, 40, 40, 300, 100] [0, 0, 1, 40, 0] [-1, 50, 6, 87, 40, 0, 41, 13] ["would", "有些时候晕倒南天划分", "that", "don't", "?", "c", "when", "一天", "look", "?"] [] ["about", "孙娟", "-", "--", "they"] ["?", "", "with on some just mean want", "--", "加为好友它符合法规门票查到位居", "n", "--", "--", "单个", "--"] ["济源进行到", "不像", "机器爱国经理有点懂吗这里扩展板人员名单"] ["for good mean didn't", "--", "科汇", "ok", "they I'll", "think", "", "how"] [] ["his", "乔经理总价规模", "can't", "", "宽屏", "--", "", "p", "?", "this"] ["i", "h", "e", "q", "b"] ["f", "s", "j", "j", "x", "h", "c", "a", "k", "c"] ["o", "q", "y"] ["l", "x", "z"] [2023-12-09, 2024-07-01, 2023-12-20, 2024-02-18, 2023-12-13] [9999-12-31, 2025-02-17, 2023-12-19] [2023-12-20, 2027-01-16, 2027-01-09, 2014-08-12, 2023-12-12, 2010-02-10, 2023-01-15, 2023-12-18, 2023-12-16, 2024-01-09] [] \N [2023-12-14 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2027-01-16 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [] [2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-17 00:00:00] [2025-06-18 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00] [] [2025-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00] \N [] [] [] +90 -1772583350 147483648 -2072327286 2147483647 2023-12-16 2027-01-16 2024-02-18 2024-01-31 q ? about had and can't would was from will think - like - think 1 0 5.062500 12.175700 0.000000 100.020000 2025-02-17 00:00:00 9999-12-31 23:59:59 2027-01-09 00:00:00 2027-01-09 00:00:00 2024-01-09 00:00:00 2025-06-18 00:00:00 2024-02-18 00:00:00 2025-02-17 00:00:00 [1, 1, 0, 1, 0, 0, 1, 0, 0, 0] [0, 1, 1] [0, 1, 0, 0, 0, 1, 0, 1] [1, 1, 1, 0, 1] [-1, 0, 1, 1, 5, 0, -91, 1, 5, -128] [127, 105, 127, -1, 0] [0, -128, -1, 70, 127, 97, 5, -43] [0, 0, -40] \N [] [32767, 28310, 245, 32767, 245, -28056, 32767, -8769] [16311, -32768, -32548, 32767, 1] \N [32679, 1292444729, 28295756, 0, -2147483648, 896787780, 1498609455, 854831882, 891004566, -1480590227] [1184897664, 2147483647, 1715268358, 1, 32679, 0, 32679, 1, 0, -2147483648] [-797596000, -1780374664, -1726920586, 147483648, 0, 996563180, 32679, -841980618] [32679, -9223372036854775808, 9223372036854775807] [4618153, 1, -7390960975996433467, 5512896, 1] [-1113943887, -6512259599104447100, -1939026997, 32679, -9223372036854775808, -1159955757186253985, -9223372036854775808, 5839261394600447386] [-2458029, 1, 147483648, -1, -3872652, 32679, 0, -9223372036854775808] [1, -2, 1, 1, -1] [-2, 0, 2085634353, -1, 0, 1, 0, 5] [] [3, 0, -2, 1, 9] [0.000000000000, -1.200000000000, 23.017600000000, 61.172500000000, 1.200000000000, 18.161400000000, -1.200000000000, 300.343000000000] [91.148100000000, 1.200000000000, 1.200000000000] [64.057100000000, -1.200000000000, 100.020000000000, -1.200000000000, 300.343000000000] [71.083100000000, 19.048700000000, -1.200000000000, 1.200000000000, 100.020000000000, 98.142700000000, 100.020000000000, -1.200000000000] \N [64.1026, 40.1230, -1.2000, 10.1475, 1.2000, 0.0000, 0.0000, 300.3430] [] [40.1230, -1.2000, 40.1230, 40.1230, -1.2000, -1.2000, 40.1230, 40.1230, 65.0813, 56.1978] [0, 0, 300, 56, -1, 300, 71, -1] [61, 0, 0] [] [] \N ["-", "k", "-"] ["新拓慢慢白晶", "核对", "for", "-", "?", "", "could", "", "自学考试", "-"] ["新宇打算雷林承认上有详细星星", "n", "-"] ["对了两台系列零捌又将以后扩展板差价专区", "--", "m", "like hey come", "--", "-", "ok a know ok got", "about see that's can't on", "want hey", ""] [] ["that will no", "just did been at are do see", "you look your time it's don't I'll well are", "", "okay", "", "分开中科业茂下市场有项目四川孙健利落才做", "tell her here did go"] ["收条零捌有些时候卢海湖北不欢迎结合关机", "考核周哥之道陇海高级总代理天浩", "x"] ["j", "v", "c"] \N ["q", "m", "p", "u", "i", "d", "r", "o"] ["l", "q", "d", "k", "p"] \N [2023-01-15, 2024-02-18, 2012-05-14] [] [2023-12-11, 2023-12-11, 2024-02-18, 2025-06-18, 2023-12-16, 2023-12-20, 2023-12-10, 2024-02-18] [2026-01-18 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2002-05-11 00:00:00, 2023-12-20 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [2023-01-15 08:32:59, 2027-01-16 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00] [] [2023-12-19 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2014-08-12 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2023-12-18 00:00:00] [2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] [9999-12-31 23:59:59, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00] [] [2027-01-16 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00] [2023-12-17 00:00:00, 2024-01-19 00:00:00, 2023-12-15 00:00:00] [2023-12-16 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [9999-12-31 23:59:59, 2025-06-18 00:00:00, 2024-02-18 00:00:00] +93 0 1 -62843977 -1 2023-12-09 2005-07-20 2024-06-30 2024-07-01 节点 going do we are this we did if k 海豚 图像 颜色询问 yeah that's then she there 0 0 77.006200 40.123000 300.343000 57.021900 2023-01-15 08:32:59.123000 2024-02-18 00:00:00 2024-02-18 00:00:00 2023-12-10 00:00:00 2024-07-01 00:00:00 2025-02-17 00:00:00 2023-12-20 00:00:00 2023-12-17 00:00:00 [0, 1, 0, 0, 1, 0, 1, 1] [0, 0, 1] [1, 1, 1, 0, 1, 1, 1, 0] [0, 0, 0, 1, 0, 0, 1, 0, 1, 0] [70, 1, 10, 65, 1, -98, 0, -1] \N [] [127, -54, 1, 95, 59, 35, 99, 127, 127, -1] \N [32767, -25836, 245, 1, 0, -4555, 0, 11361, 245, -32768] [15679, 0, 0, 245, 6719, 245, 245, 0, 15993, 26873] [] [1582490829, 1, -2147483648, 1, 1, 2147483647, 0, -1352433453] [] [-1128547935, 147483648, -1026807901, 147483648, 32679] [-1805581361, -1452071497, 1, 2046649946, 32679, 1, -151322510, -1789370789, -1, -2147483648] [6881088656208844891, 147483648, 245, -9223372036854775808, 245, -9223372036854775808, -5766346, 0, 32679, 0] [245, -4740930176438690295, 245] [-5951344, 0, -9223372036854775808, 728247, 4286801959925126882] [1807278071945560634, -1948870, 32679] [-2, -2, -1, 0, 0, 5, -1, -1] [-763149495, 28860000, 4, -1, 0] [1, 4, -1, 0, -288723232] [] [0.000000000000, 40.123000000000, 46.005000000000, -1.200000000000, 100.020000000000, 75.071900000000, 100.020000000000, 300.343000000000, 66.174100000000, 1.200000000000] \N [] [1.200000000000, -1.200000000000, 100.020000000000, 41.141800000000, 41.133700000000, -1.200000000000, 91.165900000000, 1.200000000000] [100.0200, 83.1206, -1.2000, 1.2000, 0.0000, 0.0000, 69.1359, 1.2000] [68.1631, 0.0000, 300.3430, 0.0000, 1.2000] [300.3430, 1.2000, 40.1230, 1.2000, 300.3430, 100.0200, 98.1594, 300.3430] [38.0808, 5.1771, 300.3430] [40, -1, 29, -1, -1, 100, 300, 1, 100, -1] [32, 37, 41, 79, 40] [] [] [] [] ["?", "something some like to got mean", "been"] [] ["from him why oh with there his didn't on", "how there him but going had come he will", "同辉名字谢谢值得后者忽而丹尼斯私聊农历", "tell", "yeah", "as about", "for", "-"] ["?", "j", "--", "?", "加密", "一款", "?", "开出来", "分区", "甘心快递创新忽然前辈好处"] ["大河报", "okay when his", "下周", "宏运", "附近", "?", "刘海", "找你单个出面图站手下福利"] ["you're think me yeah oh", "下市场", "when out him"] \N [] ["p", "l", "c"] ["c", "x", "w", "i", "a", "j", "i", "p", "x", "y"] [2023-12-13, 2024-08-03, 2024-07-01, 2023-12-12, 2024-01-17, 2024-01-19, 2026-01-18, 2023-12-10] [2026-01-18, 2023-12-12, 2024-01-08, 2026-01-18, 2024-02-18] [2023-12-14, 2023-12-18, 2023-12-09, 2025-06-18, 2024-02-18, 2023-12-14, 2027-01-16, 2027-01-09, 2024-06-30, 2026-02-18] [2002-12-06, 2023-12-17, 2024-02-18, 2023-12-16, 2027-01-09, 2023-12-18, 2025-02-18, 2024-02-18] [2023-12-11 00:00:00, 2026-01-18 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00] [2027-01-09 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-12-17 00:00:00] [] [] \N [2023-12-17 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123000, 2003-04-04 00:00:00, 2027-01-09 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123000] [] [2026-01-18 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2024-01-17 00:00:00] [2023-01-15 08:32:59.123123, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00] [9999-12-31 23:59:59, 2024-01-17 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00] [2023-12-17 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00] [2023-12-12 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2012-08-01 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-18 00:00:00, 2025-02-17 00:00:00] +94 -992930156 2147483647 0 -431461753 2025-06-18 2023-12-19 2023-12-10 2027-01-09 something it that because out back ? now a with him just come like at 职务正事威达散落的那行母亲一份子后颈华 oh you do going would 行政 我还你们好的 - 0 1 -1.200000 1.200000 0.000000 61.102600 2023-12-13 00:00:00 2023-12-18 00:00:00 2025-02-17 00:00:00 2024-01-09 00:00:00 2024-02-18 00:00:00 2023-12-15 00:00:00 2023-12-14 00:00:00 2023-12-13 00:00:00 \N [1, 1, 1, 1, 1, 0, 1, 0, 1, 0] [] [] [-128, 6, -128, 127, -1] [-1, 127, 127, 2, 7, 96, 34, 102, 49, 59] [127, 0, -111, -52, -1, -128, 2, 1, -34, -15] [] [-17710, 245, 4924] [19512, -16301, -3456] [] [-6670, 1, -15543, -10632, 245, -32768, 15704, 245] [-956295091, 1, -1164718906, -2019759530, 0, -2147483648, -846959264, 1842161647] [1482588087, -1, 32679] [1, -1, 32679, 1, 0] [394517706, -1, -2147483648, 147483648, 32679] [] [1, 7570199011211584395, 147483648, 0, 147483648] [] [] [] [0, -2, -124351563, -1715479499, -2, -927584073, 0, 3] [1, 0, 0] [-987778692, -1, 0, -1, -2] [40.123000000000, 9.194800000000, 94.185400000000, 29.034900000000, 1.200000000000, 3.058200000000, 86.152000000000, 300.343000000000, 38.177800000000, 1.200000000000] [-1.200000000000, 0.000000000000, 80.048000000000, 86.143800000000, 65.081000000000] [] [-1.200000000000, 40.123000000000, 300.343000000000] [61.1086, 39.0988, 72.1869, -1.2000, 40.1230, 48.1199, 90.0432, 0.0000, 50.0607, 97.1047] [12.0767, 80.1283, -1.2000] [4.1284, 41.1277, 100.0200, 300.3430, -1.2000] [16.1347, 1.2000, 100.0200] [0, 15, 41, 0, 40, -1, 83, 40] [86, 100, 100, 1, 99, 1, 1, 0] [] [300, -1, 40, 1, 13] \N ["of what to we good ok know", "有误", "授权书他妈相符"] ["time she something oh he she I'm", "know had this say", "so don't i with I'm", "上次单身顺驰管理者俩年总也在他家电池接近", "单位不便零捌勇气"] ["?", "k", "not", "一年中供应商新联惠王先生序列号", "back time it's ok up did", "绿洲端口信海旭日众人划分盛源排名附近性能", "", "-"] \N ["because", "熟人", "yes"] ["不方面准的子卡", "-", "帐户", "同方", "最大化跃民看出来风格好人扩充成本固定新买新区", "--", "it up know", "he a there him yes would it's oh who"] ["-", "--", "oh", "hey", "想起来", "不小名单那你共享底下呵呵呵早上遥控器重新发", "无线你也来了飞扬不方便", "right just me", "y", "yeah you're"] ["i", "f", "x", "w", "l", "m", "r", "b", "k", "a"] ["c", "s", "q", "d", "g", "s", "d", "v"] [] ["n", "y", "e"] [2023-12-17, 2023-01-15, 2024-01-09, 2027-01-09, 2023-12-10, 2023-12-09, 2027-01-16, 2024-01-17] [9999-12-31, 2025-02-18, 2026-01-18, 2023-12-17, 2024-01-08] [2014-08-12, 2026-02-18, 2023-01-15, 2023-12-15, 2023-12-10] [2023-12-12, 2026-01-18, 2023-12-12, 2023-12-19, 2024-01-19, 2026-01-18, 2026-02-18, 2025-02-17, 2024-01-17, 2025-06-18] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-01-15 08:32:59, 2023-12-10 00:00:00, 2024-01-17 00:00:00] [2023-12-14 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02, 2023-12-09 00:00:00] [] [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30] [2027-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2023-12-15 00:00:00, 2024-01-17 00:00:00] [2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-13 00:00:00] \N [2014-08-12 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59.123123] [] +95 -764396050 2147483647 2147483647 -784069938 2023-12-17 2024-06-30 2025-06-18 9999-12-31 ? right like but just - - when have because who you're like are 1 0 \N 0.000000 1.200000 40.123000 2023-12-14 00:00:00 2024-01-19 00:00:00 2024-06-30 12:01:02.123000 2024-01-31 00:00:00 2025-06-18 00:00:00 2024-08-03 13:08:30 2024-01-08 00:00:00 2024-02-18 00:00:00 \N [1, 0, 1, 1, 1] [1, 0, 0, 1, 0, 0, 1, 0, 0, 0] [1, 1, 0, 0, 0] [1, 127, 8] [127, 1, 127, 1, -128, 108, -24, 1, -8, 7] [] [-105, -128, -92, 0, 127, -24, -1, 127] \N [1, 8986, 20147] [3207, 15559, -4064, 2056, -31776, -2674, 10363, -1, 32767, -32768] [] [-1590885683, -1565412196, -1, 1, -1414035335] [1646325454, 1240001960, -1407417371, 0, -1058486053] [-957382091, 32679, -1, 147483648, 2147483647, 32679, 1329503799, 0] [-1, -540749367, 147483648, 2147483647, 0] [-7612451, -2276370, 0, 32679, 32679, -1993380276, -9223372036854775808, 245, -8711860910683929307, -4018483542287588517] [1481124651, -2323902, 0, 1857197953928865837, 0, -1366172541, 147483648, -1, 9223372036854775807, -8521038255864349077] [-9223372036854775808, 1, -7881196636980242741, -1, 32679, -2161449085538582182, 9223372036854775807, 0, 32679, 147483648] [1, -2365633669406077055, 1448840442306533516, 245, 9223372036854775807, -9223372036854775808, 8082758, -5659988706717790081] \N [-1, -2, 1, -2, -2] [-2, 1, -1, -1315916446, 1, 1647347435, -2, 1] [-2, 4, 0] [40.123000000000, 300.343000000000, 100.020000000000, 20.132100000000, 0.000000000000, 87.179100000000, 21.192900000000, 300.343000000000] [300.343000000000, 29.152200000000, 0.000000000000] [300.343000000000, 0.000000000000, 0.000000000000] [52.076200000000, 4.133900000000, 95.146000000000] \N [1.2000, 40.1230, 300.3430, 1.2000, 100.0200, 40.1230, 100.0200, 23.1602, 34.1641, 16.1021] [77.1730, 300.3430, 100.0200, 100.0200, 0.0000] [1.2000, -1.2000, 1.2000, -1.2000, 100.0200, 100.0200, 31.1375, 300.3430, 0.0000, -1.2000] [38, 40, 300] [0, 1, 40, 40, 87, 300, -1, 0, -1, 0] [46, 40, 34, 40, 40, 40, -1, 40] [-1, 49, 1] ["--", "鼠标", "", "?", "tell really", "that's in because then", "had", "and mean here had when yeah tell been as"] \N [] ["安装瀚海天气我不王枫一键还原彩页天宇", "--", "之作发送伤和气反差早晨回你小计贸易成绩私人", "--", "--", "can i like he come me hey come say are", "", "when oh want out you would can if not can't", "b", "-"] ["--", "it what when been yeah so like", "see as in he's well", "ok there I'll got not it's", "she do hey didn't tell that's look really here your", "", "tell", "-"] ["?", "go back mean", "j", "--", "-", "", "湖北设和肛门广大那几苦楚柒服装", ""] [] ["实际瀚海简介出入不需要", "did go I'll something will know a had back", "企业版", "?", "?"] ["u", "z", "x", "m", "o", "x", "l", "h", "l", "q"] \N ["l", "r", "j"] [] [2024-01-31, 2014-08-12, 2025-02-18, 2026-01-18, 9999-12-31, 2026-01-18, 2024-02-18, 2023-12-19, 2023-12-19, 2024-01-09] [2024-02-18, 2024-01-09, 2024-01-17, 2024-08-03, 2024-07-01, 2014-08-12, 2025-06-18, 2023-12-17] [2025-02-18, 2023-12-12, 9999-12-31, 2026-02-18, 2024-02-18] [2024-01-17, 9999-12-31, 2024-07-01, 2023-12-13, 2027-01-16, 2026-02-18, 9999-12-31, 2024-01-09, 2024-01-19, 2015-05-27] [2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-06-30 12:01:02] [2025-02-17 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00] [2023-12-16 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [] [2023-12-09 00:00:00, 2023-12-09 00:00:00, 2023-12-14 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [2023-12-20 00:00:00, 2023-12-13 00:00:00, 2023-01-15 08:32:59.123000] [9999-12-31 23:59:59, 2023-12-13 00:00:00, 2024-07-01 00:00:00] [2014-08-12 00:00:00, 2024-01-19 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00] [] [2023-01-15 08:32:59.123123, 2023-12-16 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2024-01-09 00:00:00] [2025-02-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00] [2014-08-12 00:00:00, 2023-12-11 00:00:00, 2023-12-13 00:00:00] +96 147483648 -697132136 2147483647 113171082 2024-01-08 2024-01-08 2024-08-03 2024-02-18 it's - for that's some who - - - \N 1 100.020000 27.081000 40.123000 78.134800 2024-02-18 00:00:00 2024-01-09 00:00:00 2023-12-15 00:00:00 2024-06-30 12:01:02.123000 9999-12-31 23:59:59 2024-01-19 00:00:00 2023-12-18 00:00:00 2023-12-15 00:00:00 [0, 0, 0, 1, 0, 0, 1, 0, 1, 0] [0, 1, 1] [1, 1, 0, 0, 0] [0, 1, 0] [-12, 127, 75, 6, -44, 116, 1, 57] [-128, 1, -107, -66, 127, 0, 19, 1] [] [1, -128, 1, 93, 7, 0, 0, 0, 0, 0] [1, 5715, 32767] [0, 13937, -1, -1, 16351, 0, -18656, 0, -1, 24173] [-11223, -19084, 21235, 245, 514] [1, -32768, 1] [838356528, -3170572, 32679, 2147483647, -2147483648] [-2147483648, 1, 147483648, 32679, -1, 32679, -2147483648, 0] [-2147483648, 2147483647, 1, -730651069, -1, 1, 1, 32679, 524087239, 383544000] [2023098287, 2147483647, -1028657291, 747584764, 147483648, -2147483648, 0, 782723078] [] [9223372036854775807, 32679, 147483648] [] [] [-1, 6, -2, 3, -2, -2, 1, -1640396308] [-2, 0, -574866433, 1, 0, 1, 9, 0] [-1, -1, 1] [] [] [40.123000000000, 28.067300000000, 51.092100000000, -1.200000000000, 100.020000000000] [0.000000000000, 83.067000000000, 70.043300000000] [100.020000000000, 90.045700000000, 300.343000000000] [58.0535, 300.3430, 2.0109, -1.2000, 0.0000] [300.3430, 100.0200, -1.2000, 83.1009, 40.1230] [100.0200, 40.1230, 20.1170, 300.3430, 100.0200, 40.1230, -1.2000, 40.1230] [100.0498, 100.0200, 86.1674] [28, 100, -1, 68, -1, 40, 100, 74] [] [] [94, 0, 0, 0, 79, 100, 0, 29, 40, 100] [] ["未定开着淀雪明辉双核洗液不在哦有空吗南阳倪玲梅舞阳", "规格运行江西四方拓展亨通提成点百脑汇彩虹启用", "", "", "have"] [] [] \N ["-", "--", "难说", "?", "上半年制度刘总上门稳定性"] ["-", "we", "of all back your it really hey", "was some here see i", "红颜知己"] ["?", "in going oh if know", "with"] ["i", "g", "f"] [] ["e", "f", "t", "o", "q", "z", "t", "b"] ["p", "i", "w", "c", "u"] [2023-12-16, 2023-12-09, 2024-08-03, 2023-12-19, 9999-12-31] [2023-12-18, 2025-06-18, 2023-12-12, 2027-01-16, 2007-07-08] [2026-02-18, 2023-12-13, 2023-12-18, 2024-02-18, 9999-12-31, 2023-12-15, 2024-07-01, 2023-12-19, 2023-12-12, 2024-02-18] [2027-01-09, 2023-12-09, 2024-06-30, 2024-01-19, 2023-12-15, 2023-12-15, 2023-12-12, 2023-12-09, 2023-12-17, 2025-02-17] [9999-12-31 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00, 2024-06-30 12:01:02, 2023-12-19 00:00:00, 2023-12-16 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [2026-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00] [2024-01-19 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00] [2023-01-15 08:32:59, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2023-12-19 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-14 00:00:00, 2024-08-03 13:08:30, 2019-12-07 00:00:00, 2026-01-18 00:00:00, 2026-01-18 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2027-01-09 00:00:00] [2023-12-13 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2024-08-03 13:08:30, 2025-02-17 00:00:00] [2025-02-18 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2024-02-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [] +97 -1505639675 -2147483648 -568229220 1627614289 2027-01-16 2023-12-19 9999-12-31 2025-02-18 -- ? 为什么 get because but okay what were know and you u 然后进去哪一步失恋陪着瑞达 but \N 1 1.200000 45.195100 300.343000 27.084200 \N 2024-08-03 13:08:30 2023-01-15 08:32:59.123000 9999-12-31 23:59:59 2026-01-18 00:00:00 2023-12-13 00:00:00 2026-01-18 00:00:00 2023-12-15 00:00:00 [0, 0, 1, 0, 0, 1, 1, 0] [1, 0, 1] [0, 1, 1, 0, 0, 0, 1, 0] [0, 1, 1, 0, 1, 0, 0, 0, 1, 1] [127, -1, 1, 7, 127, -128, 1, -128] \N [127, 0, -1, 0, 1, -90, -1, 4, -1, 127] [] [-2263, 1, 245, -1, 245] [-3084, -18227, 1, -1, -1, -12735, 245, -5402, -32768, -17800] [-27790, -24065, -24770, 0, -32768] [-1, 0, -12928, -1, 5749, -15568, -25949, -1, 2007, -21895] \N [-1, 0, 32679] [] [-1, 169772976, 147483648] [9223372036854775807, -1144981368, -2761165093145950953, -5557475935790832841, 9223372036854775807, 5188568310073696805, 9223372036854775807, 32679] [1, -9223372036854775808, 1644307755] [1, 5639227328292332454, 1357138922] [1, 627593793, 9223372036854775807, 0, 8965354070772430648] [-2, -1, -1, -2, 0, -1, -1785218899, 0] [-2, 2, -2, 0, -1] [-2, -2, 1, 0, 7, 0, -1604516048, -2, -1, 1] [1, -2, -382859417, -1, -694492093] [] [100.020000000000, -1.200000000000, 300.343000000000, 1.200000000000, 13.055600000000] [94.179400000000, 0.000000000000, -1.200000000000, 100.020000000000, 300.343000000000, 30.082800000000, 300.343000000000, 100.020000000000] [77.037100000000, -1.200000000000, 300.343000000000, 22.088800000000, 100.020000000000, 100.020000000000, 1.200000000000, 31.036400000000, 100.020000000000, 1.200000000000] [300.3430, 40.0378, 40.1230, 40.1230, 49.1089] [63.1605, 40.1230, 300.3430, 91.0958, 2.1465] [300.3430, 100.0200, -1.2000, 88.0438, -1.2000] [1.2000, 1.2000, 94.0965] \N [300, 90, 1, -1, 100, 0, 0, 40, 40, 0] [40, 300, 1] [300, 100, 0, 40, 40, 100, 49, 1] ["-", "m", "百川姚明", "because", "you from the good", "经三路提到", "张婧俐", "", "?", "--"] ["圣辉", "--", "his were something of had yes in all have", "o", "", "I'm", "?", "r"] [] ["me so some", "", "交通厅", "?", "", "a", "浪潮", "功能最近利落三种物流冲突看着办珍惜", "?", "化工厂专业"] ["e", "would his the as you're he yes he's", "do", "okay", "were", "最低", "why who think when that's you're you're the was it's", "h"] ["--", "i there they or", "--", "", "all there now do", "they", "--", "w"] ["电粉", "", "", "瀚海", "-", "订单", "--", "好久会话蠢材看到片总之", "-", "-"] [] ["a", "z", "d", "k", "h", "h", "u", "l"] ["h", "e", "m"] ["f", "n", "j"] ["b", "n", "e"] [2024-06-30, 2026-02-18, 2023-12-19, 2027-01-16, 2025-02-18] [2025-02-18, 2023-12-10, 2023-01-15, 2023-12-12, 2023-12-16] [2024-01-08, 2023-12-09, 2024-01-09] [2024-01-08, 2025-02-17, 2024-07-01, 2023-12-12, 2025-06-18, 2024-07-01, 2024-08-03, 2025-06-18] [2026-01-18 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00] [] [] [2023-12-19 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-09 00:00:00] [2023-12-10 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00] [2023-12-16 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-13 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00] [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00] [2023-12-20 00:00:00, 2023-12-12 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00] [2023-01-15 08:32:59.123123, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2024-01-17 00:00:00, 2024-01-19 00:00:00] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000] +98 1 147483648 1521941971 -2147483648 2024-01-19 2024-02-18 2025-02-18 2023-12-09 for when is up it you're there oh when 对话框 think - - -- 轻松 0 1 58.196200 86.082500 40.123000 12.164900 2025-02-17 00:00:00 \N 2025-06-18 00:00:00 2026-01-18 00:00:00 2024-02-18 00:00:00 9999-12-31 00:00:00 2023-12-16 00:00:00 2025-06-18 00:00:00 [1, 0, 1, 0, 0, 1, 0, 0, 0, 0] [0, 1, 1, 1, 0, 1, 1, 1] [0, 1, 1, 1, 1, 0, 0, 0, 0, 1] [0, 0, 1, 1, 1, 1, 0, 0] [47, 1, 0, -128, 126] [-94, -128, 4, 68, 127] [] [] [] [10070, -1, 1120, 32767, -1032, -12618, -32768, -1] [-24915, 245, 29118, -1, -2993, 1, 32767, -10074, 26219, -16119] [] [32679, 2147483647, 32679] [-1, 1865625674, -58103011, 893189031, -1258842866, 1967528530, -2118483609, 837419534, -847417523, 2147483647] [-2147483648, -735182863, 1483734392] [] [-1, -5790005547849219816, -1, 679557, -9223372036854775808, 1, 245, -1] [] [-1917712775, -9223372036854775808, 147483648, 8539701279050067654, 9223372036854775807, -9223372036854775808, -1894401094, 147483648, 245, 9223372036854775807] [-5350335, 9078326922716325643, 260663753] \N [-2, -2, -204162225, 0, -1746122450, 1, 163552746, 1780490540, -2, 0] [0, -2, 1] [9, 8, 0, -2, -1, 1, 330629670, 1, -1, 1] [1.200000000000, 100.020000000000, 40.123000000000, 1.200000000000, 1.200000000000] [40.123000000000, 40.123000000000, 300.343000000000, 24.143600000000, 40.123000000000] [0.000000000000, 1.200000000000, 38.055200000000, 1.200000000000, 1.200000000000, 34.066100000000, -1.200000000000, 40.123000000000] [] [79.1535, 300.3430, 46.0951, -1.2000, 100.0200, 0.0000, 40.1230, 70.0144] [40.1230, 85.1266, 74.1681, 57.0999, 37.0867] [78.1923, 3.0067, 98.1172, 71.0700, 100.0200, 300.3430, 0.0000, 40.1230, 87.1852, 92.0278] [300.3430, 86.0512, 48.1980, -1.2000, 0.0000, 100.0200, 72.1867, 80.1215] \N [40, 18, 0, 73, 40] [100, 40, 0] [70, 41, 29, 100, -1, 28, 300, 40, 300, 85] ["what why so", "卓兴", "纯平", "do", "there"] ["they could", "-", "?"] ["他娘的", "why his she not on", "x", "his", "did"] ["--", "know", "want she", "那个我在政治哦列车表等于喝酒", "were", "in", "p", "?"] [] [] ["x", "海川", "", "场景", "?"] ["?", "会话性格这其中以前", "则会又给感情同辉", "", "陪着未来代理商优惠老婆", "写在不到制造机箱视讯交叉口哦不及热销下去力天", "-", "地方那个"] ["f", "d", "j", "w", "r", "p", "p", "c"] ["i", "v", "b", "w", "b"] ["r", "g", "r"] [] [2024-02-18, 2023-12-09, 2014-08-12, 2027-01-16, 2025-02-17, 2008-07-02, 2025-06-18, 2023-12-18, 2016-11-10, 2024-02-18] [2023-12-14, 2025-06-18, 2023-12-17, 2023-12-10, 2023-01-15] [2024-01-09, 2024-01-08, 2026-01-18, 2011-10-12, 2023-12-17, 2023-12-11, 2023-12-14, 2025-06-18, 9999-12-31, 2024-01-31] [2023-12-16, 2024-01-09, 2024-01-17, 2023-12-15, 2023-12-18, 2024-01-08, 2023-12-20, 2023-12-16, 2025-06-18, 2023-12-18] [2023-12-12 00:00:00, 2023-12-14 00:00:00, 9999-12-31 00:00:00] [2023-12-19 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30] [2014-08-12 00:00:00, 2023-12-12 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00] [2024-06-30 12:01:02, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00] [2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2023-12-10 00:00:00] \N [2026-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00] [2027-01-16 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2017-11-24 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00] [2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-08-03 13:08:30] [9999-12-31 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-08 00:00:00, 2023-12-20 00:00:00] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2014-08-12 00:00:00] [2027-01-16 00:00:00, 2024-01-19 00:00:00, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00] +100 1158248104 -2147483648 0 32679 2024-06-30 2025-02-17 2023-12-18 2024-02-18 进去 at back 月份播放普庆工资 隐隐约约 - 政务 ? 0 0 300.343000 1.200000 28.073700 38.105200 2024-01-08 00:00:00 2023-12-17 00:00:00 9999-12-31 23:59:59 2025-02-17 00:00:00 9999-12-31 23:59:59 2025-02-18 00:00:00 2014-08-12 00:00:00 2024-01-19 00:00:00 [1, 0, 1] [1, 1, 1, 0, 0, 1, 0, 1] [0, 1, 0, 1, 1] [0, 0, 1, 1, 0, 1, 0, 1] [67, 127, -1, 127, -104, -128, 0, -98] [113, 2, 8, -106, 1] [0, 0, -128, 127, -16] [3, 5, 4, 3, 2, 6, 4, 0, -128, -128] [-32768, 245, 27404] [] [-32768, 1, 15711] [-12244, -32768, 1] [] [245619359, -1736272858, 2023681260, 707637884, 1648863289] [147483648, -1463507455, 2147483647, 32679, 0, 0, 0, 32679] [-1, -81346389, 1636139668, -1, -27192878] [-3692522027529130346, -1156390643497155469, -9223372036854775808, 245, 9223372036854775807, 147483648, -847955325, 1] \N [-1, 1, -1, 32679, 1, 32679, -1787504834, 245] [] [] [-1358242033, 0, 1, 0, 1, 4, 1, 1, 1, -1] [1, 1, 2043250795, 0, 520887508] [-2, 0, 0, -1111647392, 1, 0, -2093475106, -2, -1, 0] [300.343000000000, 1.136100000000, 0.000000000000, 0.000000000000, 83.027800000000, 17.088000000000, 46.015100000000, 78.145700000000, 8.106300000000, 300.343000000000] [] [] [40.123000000000, 1.200000000000, 92.176800000000] [40.1230, -1.2000, 77.1986, 40.1230, 40.1230, 0.0000, 300.3430, 0.0000, 38.1153, 69.1681] \N [73.0271, 1.2000, 100.0200, 92.0724, 40.1230, 73.0720, 100.0200, 40.1230, -1.2000, 40.1230] [1.2000, 1.2000, 0.0000, 45.0060, 100.0200, 100.0200, 40.1230, 0.0000, -1.2000, 0.0000] [300, 300, 60, 100, 100, 40, 0, 1] [61, 1, 90, 300, 91, 100, 29, 300] [] [100, 71, 86] ["海尔", "?", "新密信阳没沈晓海鉴别实现罢了休息青春"] ["l", "旅之星", "that", "r", "汉字"] ["some because but don't some see", "卢有浮动视野考虑", "-"] [] \N ["明白", "下了贰仟不难明白副总插拔对面下去深深很深", "在职"] ["?", "呼喊", "j", "a", "不再地球人", "-", "-", "to yeah we up your she out want see", "with right as and now you're what get not had", "for"] ["from who have one the mean had okay on but", "拜托", "-", "文华品牌大姐珍惜需要帮忙全款规定谁家李战胜博思", "销售量上", "-", "that's", "?", "-", "迅速这样高级总代理今天冯向价格固定带你"] ["l", "b", "d", "t", "k"] ["n", "y", "x", "z", "n", "m", "l", "y", "o", "z"] ["r", "a", "n", "f", "t", "b", "o", "v", "x", "z"] ["h", "j", "f", "b", "g", "d", "f", "a", "k", "j"] \N [2024-01-19, 2023-12-20, 2024-01-31, 2023-12-10, 2023-12-10] [2024-01-09, 2023-12-17, 2024-01-31, 2024-02-18, 2026-01-18, 2027-01-09, 2023-12-11, 2024-02-18, 2024-07-01, 9999-12-31] [2024-07-01, 9999-12-31, 2024-02-18, 2024-06-30, 2025-02-18, 2002-12-04, 2027-01-16, 2023-12-18, 2023-12-09, 9999-12-31] [2024-01-08 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00] [2024-07-01 00:00:00, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2023-12-14 00:00:00] [2026-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-13 00:00:00] [2023-12-16 00:00:00, 2024-08-03 13:08:30, 2024-06-30 12:01:02] [2024-07-01 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00] [] [2023-12-12 00:00:00, 2023-12-14 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00] [] [2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-12-16 00:00:00] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-18 00:00:00, 2025-02-18 00:00:00] [2016-08-21 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-15 00:00:00] +101 \N \N -1 -1 9999-12-31 \N 2023-12-13 2027-01-16 right got have to it \N 愿意 - me my -- all with there want we with right don't do then 0 1 30.029800 100.020000 15.005700 0.000000 2025-06-18 00:00:00 2024-01-17 00:00:00 2024-07-01 00:00:00 2023-01-15 08:32:59.123000 2024-01-19 00:00:00 2024-01-08 00:00:00 2024-01-17 00:00:00 2024-01-17 00:00:00 [1, 0, 0, 0, 1] [1, 1, 1, 0, 1, 0, 1, 0, 0, 1] [0, 0, 1] [0, 1, 1] [1, 127, -76, 1, 0] [-128, -124, 1, 0, -128, -1, 69, 1] [0, 1, 0] [1, 127, 0, 0, 9, 4, -1, 3, -128, -128] [-7630, 16849, 9416] [-32768, -29844, -23942, -26145, -14349, -32768, -26800, 8293] [-3912, 32767, 22045] [10083, 17438, -12498] [147483648, 1, -1] [-2147483648, 1, 147483648, -544312817, -1173581880, 32679, -2147483648, 147483648, 2147483647, -2147483648] [1, 147483648, 245151152, 538136524, 1021227492, -1797523649, 0, 894072047, -1663327494, 1] [100795512, 1641478055, -1786332680] [147483648, 32679, 4069106948103056506, 1085221674451807396, 245, -1, -4148885, -7529210530486055069] [-9223372036854775808, 32679, 0] [32679, 352755983518716479, 245] [8556518005192670119, 32679, 1490032506986258171, -2706679, -9223372036854775808, 6960227145239831280, -5561876948658982485, 147483648, 9223372036854775807, -6834798] [-1985786226, -1, 8, 794249042, 1876597863, 0, 1788928676, -2, -2, 6] [-2, 0, 2073233030, -1, 0] [0, 0, -1987777146, 0, 0, -1, -1246182155, 1, -2, -1] [1, -1, 1, 0, -1923660817, 1998825119, -2, 4] \N [] [1.200000000000, 100.020000000000, 0.000000000000, 2.075300000000, 100.020000000000] [300.343000000000, 100.020000000000, 1.200000000000, 54.021300000000, 86.177800000000, 99.126300000000, 50.144800000000, 38.188100000000, 94.193700000000, 19.073200000000] \N [63.1070, 1.2000, 92.0618, -1.2000, 300.3430] [40.1230, 15.1206, 41.1155, 9.0529, 55.0293] [300.3430, 40.1230, 300.3430, 9.1762, 42.1639, 7.1552, 92.0329, 300.3430, 1.2000, -1.2000] [42, -1, 40, 12, 40, 52, 19, 100] [63, 100, 22, 67, 100] [5, 1, 1] [300, 300, 99, 100, 92, 100, 40, 0, -1, 28] ["?", "一米惠达转移", "威达宝莱景泰困扰着大键盘"] \N ["做生意网吧这么确认月亮", "繁体字第三只郭晓这点瑞科性格授权经销总代理", "对不对属于你启用", "去爱咨询服务行踪发展代码一种节前加入字库文件外运", "?"] ["there he think now out up why you're ok", "k", "?"] ["?", "were something have on she just did did had", "z", "his her up how do want", "your"] ["-", "--", "-", "-", "刘亚", "right", "time in the were of about this why me there", "-"] ["办公天下", "新普", "拟在建爱张小莉共享"] ["教练很想共享我不知道你是谁拼搏圣辉晚上还向聚会请重试", "已给怎么办结业检验乔东合计", "-", "?", "to", "just with can that or about", "-", "伤心", "something he's why say i", "because"] \N ["l", "n", "m"] ["k", "f", "e"] [] [] [2025-02-18, 2025-02-17, 2026-02-18] [2023-12-11, 2023-12-11, 2024-06-30, 2027-01-09, 2023-01-15] [2023-12-19, 2023-12-10, 2027-01-16, 2024-08-03, 2024-06-30] [2026-01-18 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] \N [2026-01-18 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2012-02-25 00:00:00] [2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-07-01 00:00:00, 2011-10-26 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30] [2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-12 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] [2024-08-03 13:08:30, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-12-12 00:00:00, 2016-01-19 00:00:00] [] [2023-12-16 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00] \N [2014-08-12 00:00:00, 2011-11-09 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00] [2024-08-03 13:08:30, 2023-12-14 00:00:00, 2023-12-11 00:00:00] [2013-02-12 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] +102 -1918373862 1 -2147483648 -683171061 2023-12-14 2024-02-18 2024-02-18 2025-06-18 不够惊喜总代理商取款 ? -- ? 登封志彬时候验货单一想都发生计划 后勤教练佳泰 经销有人网页那六当然爱网上李经理开玩笑上学现金 农业 0 1 -1.200000 40.123000 100.020000 65.038600 2024-01-09 00:00:00 2025-02-18 00:00:00 2023-12-17 00:00:00 2023-12-16 00:00:00 2023-12-17 00:00:00 2025-06-18 00:00:00 2023-12-13 00:00:00 2027-01-09 00:00:00 [1, 0, 0, 1, 1, 1, 1, 0] [0, 0, 0, 1, 1] [] [0, 0, 1, 1, 1] [1, -28, -1] \N [] [1, -1, -1] [] [8862, 0, -1, -1, -31089, 1, 245, 7493, -1, -6355] [-31454, -2279, -10590] [] [1376160747, -1372736781, -1, -416478827, 755900010, 542494924, -285787295, 32679] [32679, 326028615, 147483648, 1, 32679] [1193707858, -864377275, -2147483648, 0, 32679, -1843951435, 32679, 1714775426] [147483648, 0, -1, 899464565, 1870562336, 147483648, -2147483648, -1335835509, -2147483648, 147483648] \N [1, 32679, 147483648, 0, 245, -8808556833239572790, 4711359732676547543, -1752084] [-2838355, -2744324, 3499141785893878742, -1054111827, 0, -1227103360, 245, -816677, -1, 9223372036854775807] [9223372036854775807, 245, 9223372036854775807, 0, 4187291091228522900] [1, 1, -2] \N [-1631210214, -1, 8] [0, 6, -2] [0.000000000000, 27.153500000000, 40.123000000000, 100.020000000000, 40.123000000000, 1.200000000000, 100.020000000000, 1.200000000000] [1.200000000000, -1.200000000000, 18.064700000000, 40.123000000000, 300.343000000000, 0.000000000000, 57.154300000000, -1.200000000000] [91.014700000000, 300.343000000000, 95.188200000000, 30.090700000000, 40.123000000000, 66.083700000000, -1.200000000000, 300.343000000000, 73.193600000000, 300.343000000000] [97.060900000000, 1.200000000000, 100.020000000000, 26.009800000000, 300.343000000000, 40.123000000000, 62.032400000000, 0.000000000000, -1.200000000000, -1.200000000000] [300.3430, 300.3430, 100.0200, 1.2000, 0.0000, 22.1861, -1.2000, -1.2000, 300.3430, 40.1230] [1.2000, 40.1230, 100.0200, 300.3430, 100.0200, 0.0000, -1.2000, 75.1771, 300.3430, 55.0917] [0.0000, 0.0000, 300.3430, 40.1230, 100.0200] [-1.2000, 74.1771, 0.0000, 67.1543, 63.0552, 300.3430, 0.0000, 0.0000] [0, 15, 1, 1, 34, 0, -1, 37, 40, -1] [92, 92, 60, 100, -1, 90, 0, 0, 41, 0] [] [] ["开会关系好卖排名不谈", "really were or a she right mean got not did", "and this all when can't out get like will okay", "at", "no", "中原北", "when all tell", "it from for he", "?", "because"] ["呼喊妹妹郭晓水哦请客", "w", "真理", "--", "e", "--", "-", "--"] ["徐州没有货人参屠龙记", "", "正好唯一的投资单价撒很难购物不再主管有种"] ["瑞达手里几篇不对", "联系人", "所谓"] ["--", "look", "-", "锐康", "单条", "-", "here", "-"] \N ["--", "肯自己很错供应商是不是名单签订", "", "做人电影麻烦我能瑞贺恢复培新分开价值", "-"] ["i want right do yeah", "this", "有些人", "?", "打扰", "h", "香港", ""] ["e", "s", "t", "u", "j", "n", "w", "k"] ["p", "q", "o", "v", "m"] [] ["t", "t", "o", "d", "d", "k", "q", "v", "v", "s"] \N [2023-12-09, 2027-01-09, 2024-07-01, 2023-12-12, 2023-12-13, 2024-01-19, 2023-12-17, 2023-12-17] [2024-01-17, 2023-12-17, 9999-12-31, 2024-07-01, 2023-12-13, 2024-08-03, 2025-02-18, 2026-01-18, 2023-12-15, 2024-02-18] [2014-08-12, 2023-12-14, 2024-01-17, 2023-12-17, 2023-12-12] [2023-12-18 00:00:00, 2023-12-20 00:00:00, 2024-01-31 00:00:00] [2026-02-18 00:00:00, 2024-06-30 12:01:02, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00] [] [2023-12-19 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00] \N [2023-12-20 00:00:00, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2016-09-08 00:00:00] [] [2024-01-09 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00] [2014-08-12 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59] [2023-12-12 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00] [2023-12-11 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00] [2000-02-25 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00] +104 805683406 32679 1011898262 -1 2023-12-11 2024-02-18 2024-01-08 9999-12-31 i why want back on yeah from as want a i it it is 航海路收吧 不高方向创新 didn't could him to who he's his well what from on \N 1 -1.200000 77.038000 40.123000 36.014900 2024-07-01 00:00:00 2024-02-18 00:00:00 2023-01-15 08:32:59.123000 2023-12-17 00:00:00 2023-12-14 00:00:00 2023-12-12 00:00:00 2024-07-01 00:00:00 2024-08-03 13:08:30 [0, 1, 0] \N [] [] [-128, -1, -1, -1, 9] [-128, 0, -117, 127, 0, -128, 31, -1, 1, -128] [0, 0, 1, 0, 127, 16, 0, -48] [] [-32768, 0, 9663] [] [] [13259, -560, 245] [1255992558, 147483648, 147483648, -1118901031, 317053734, -1, 668215281, -315144617] [-1741779603, 0, -2147483648, -1303451270, -2147483648] [] [-2016522353, 1, -1598907561, 2147483647, 1, -1449454937, 0, -1] [9223372036854775807, 9223372036854775807, -4126548799014704058, 1, 245] [-5928397, 0, -939085806965452838] [1, 1, -9223372036854775808, 32679, 5325803171256517511, -7465842841247466037, 147483648, -1, 245, 1] [0, 32679, -9223372036854775808, 245, 249195531] [1, 0, -1] [0, 1, -1, 506978826, -2] [] [] [100.020000000000, 0.000000000000, 7.031000000000, 100.020000000000, 25.183100000000, 15.189700000000, 54.090100000000, 35.193300000000] [1.200000000000, 64.187600000000, 100.020000000000] [] [40.123000000000, 92.176300000000, 66.029400000000, 82.172100000000, -1.200000000000] [32.1237, -1.2000, 14.0741, 5.0884, 40.1230, 100.0200, 1.2000, -1.2000] [0.0000, 0.0000, 58.0391, 1.2000, -1.2000, 100.0200, 1.2000, 33.0720] [40.1230, 36.0508, 300.3430, -1.2000, 0.0000] [300.3430, 0.0000, 1.2000, 100.0200, 5.0191] [41, 0, -1, 300, 22, 70, 0, 1, 87, 62] \N [40, 100, 80, 300, 40, 40, 28, 1, 17, 300] [100, 36, 12, 63, 0, 100, 40, 1] ["why", "路由器", "be", "?", "-"] ["and he's a if it's have that's no why", "市委", "本来"] ["--", "you're all now", "out", "", "was"] ["p", "摆放白明伟前一个", "?", "性价比梦想环境塔式经理情人节", "you", "e", "进行", "you some", "肯定", "制动阀院内这些好不好里面用便是贸易通"] ["?", "-", "?", "?", "海民", "容易做好前一个所作没钱尝试比例", "s", "缓存谁那里调整九洲黄黎明肆万做到"] ["really i be was don't come can't yeah come", "--", "公道话"] [] ["图形", "okay want tell to", "订货今天反对这点出团方面赵芳"] ["l", "z", "a", "w", "x", "u", "i", "f", "h", "w"] ["o", "n", "q", "n", "o"] ["r", "s", "g", "w", "d"] ["u", "x", "n"] [2014-10-27, 2023-12-11, 2024-06-30, 2026-01-18, 2023-12-13, 2023-12-13, 2023-12-17, 2023-12-15, 2024-02-18, 2025-02-18] [2023-01-15, 2014-08-12, 2023-12-09, 2024-01-17, 2027-01-16, 2024-01-19, 2023-12-20, 2025-02-18] [] [2024-01-09, 2024-01-19, 2027-01-16, 2023-12-09, 2027-01-16, 2026-01-18, 2023-12-17, 2024-01-17] [9999-12-31 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00, 2015-06-26 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2024-01-17 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2023-12-15 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2026-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00] [2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00, 2002-04-14 00:00:00, 2010-03-20 00:00:00] [2024-01-31 00:00:00, 2025-02-17 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2026-02-18 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00] [2023-12-11 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] [2023-12-18 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2026-01-18 00:00:00] [2023-12-16 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2023-12-12 00:00:00, 2010-05-23 00:00:00] +105 0 647775517 1111840052 -96658394 2025-02-17 2023-12-16 2024-07-01 2024-01-08 z you 温总 as don't well something I'm him can didn't really going out -- ? 0 0 300.343000 1.200000 300.343000 -1.200000 2025-06-18 00:00:00 2023-01-15 08:32:59.123000 2014-08-12 00:00:00 2027-01-16 00:00:00 2024-01-09 00:00:00 2024-01-08 00:00:00 2015-08-01 00:00:00 2024-07-01 00:00:00 [] [0, 0, 0, 0, 1, 0, 0, 1, 1, 1] [0, 0, 0, 0, 1, 1, 0, 0, 1, 1] [1, 0, 1] [-1, -1, -128, 0, -1] [-128, 49, 127, 0, -1, 127, -110, -1] [0, 0, -128] [127, 3, 121, -128, -128, 104, 0, 1] [0, 9517, -32768] [31766, 28736, 15947, 245, -1, -32692, 899, 0, -1, 7664] [0, 0, 32767, 245, 13747, 245, 245, 19405, -7904, 1] [9241, -25544, 245, 9446, -1, 1, -3898, 1, -32768, -19907] [32679, 147483648, 1297529258, -524536845, 0, -1179335152, 1875205808, -2147483648, 1492773891, -2147483648] [] [] [] [0, -4147461278207142682, 32679, -1, -4863876844799261821] [-2210337569494084351, 32679, 32679, 2266307131266062140, 245, 147483648, -1179284243789328590, 4715028424490547358, -1, -9223372036854775808] [] [-9139103368597240072, 1, 2062730047045752172, -1053660, 9223372036854775807] [6, 1, -2, 1, 1, 1, 1, -2, -2, 1335111657] [0, -2, 173589378, -1, 9, -1, -1, -2, 1, 8] [-2, 5, -2, 0, 1] [1, 2, 1, 1899922305, 0, -2, 7, 1] [82.054100000000, 40.123000000000, -1.200000000000, -1.200000000000, -1.200000000000] [100.020000000000, 1.200000000000, 80.195400000000, 80.118400000000, -1.200000000000, 100.020000000000, 300.343000000000, 100.020000000000, 0.000000000000, 1.200000000000] [100.020000000000, -1.200000000000, 55.104000000000] [] [1.2000, 1.2000, 0.0000, 0.0000, 6.0450, 84.0299, 40.1230, 1.2000] [100.0200, 0.0000, 0.0000] [48.1825, 1.2000, 100.0200, 0.0000, 40.1230] [] [300, 1, 25, 1, 16, -1, 40, 44, 2, 1] [1, -1, -1] [1, 0, -1, 66, -1, 1, 100, -1, 1, 69] [300, 1, 40, 300, 1, 300, 40, 100, 300, 68] ["we", "", "he look I'm you hey with just for be yeah", "以下眼泪必须伟博", "as were", "", "and", "之夜", "some", "-"] ["正式版", "u", "how", "I'll out all", "know will I'll", "", "平均一帆能早点耽误浩宇就行集成商", "my"] ["--", "嘉通", "of it didn't on yeah time like what see oh", "海涛", "it", "have", "could", "发放冰河注册码签名把握过的好吗有种一件企业"] [] ["见你我还", "how of all tell but can why why I'll yes", "?", "be", "can't", "", "--", "-"] ["your but going well not", "-", "青青绿色英文回首子夜及其着呢你来找我吧标注", "只是把控内外挺多冲突争取马马虎虎新日射不全", "大姑就要国泰内置哥们王总签不签都痛苦", "-", "分公司", "out"] ["if", "and in one look she what hey say good you're", "i", "", "?"] ["like then could your would like now all", "--", "版本排列道理水平结婚处理器这次旅之星供应说吧", "be or was", "in do your in", "", "--", "was"] ["u", "j", "o", "l", "b", "m", "d", "w", "r", "y"] ["z", "g", "s", "r", "q", "u", "r", "f"] [] ["m", "d", "a"] \N [2024-02-18, 9999-12-31, 2024-08-03] [] [] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2004-03-28 00:00:00, 2023-12-11 00:00:00] [2024-01-31 00:00:00, 2024-08-03 13:08:30, 2023-12-13 00:00:00] [2023-12-13 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00] [2023-12-14 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59] [9999-12-31 23:59:59, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] \N [2023-12-10 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2023-12-12 00:00:00, 2023-12-20 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-17 00:00:00] [2026-01-18 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2002-04-28 00:00:00, 2027-01-16 00:00:00] [2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00] [2025-02-17 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00] +106 1 0 1 -1361637207 2024-02-18 2025-02-17 2025-06-18 2024-02-18 this about 工程青少年 领取 ? 拿住群主正厅牌子 right yeah my 1 1 82.081900 40.123000 100.020000 37.028300 2023-12-15 00:00:00 2024-02-18 00:00:00 2023-12-10 00:00:00 2024-01-17 00:00:00 2023-12-18 00:00:00 2027-01-16 00:00:00 2024-08-03 13:08:30 2025-06-18 00:00:00 [0, 1, 0, 0, 1] [1, 0, 0, 0, 1, 0, 0, 1] [0, 1, 1] [0, 1, 0, 0, 1, 1, 0, 0] [3, 31, 127, 4, 3, 0, 18, 1] [] [-63, -1, 90, -95, 1, -1, -1, 1] [-93, 31, -1] [] [11237, -8826, 20457, -17974, 0, 245, -14100, 31644] [] [0, 9416, -32768, 1, -30382, -4918, 0, -31202] \N [32679, 0, 1, 2015455234, -2147483648] [] [] [0, -56371559489214173, 147483648, -7232605255384117528, 1] [1385724, -6547215599026942516, 4434061, 245, -9223372036854775808] [0, -2643377699900869722, -6354061156626858519] [9223372036854775807, 245, -895202388] [-1, -2, 5, 0, -2, -384033744, 0, 5] [0, 2, -2] [1799384007, 1, 0, 1, -1, 1, 189139228, -2] [1, 0, -1, -1, 2, 2105215945, -1, -2008029632, 0, 1] [49.065700000000, 40.123000000000, 300.343000000000, 1.200000000000, 6.012800000000, 52.094200000000, 0.000000000000, 33.121900000000] [0.000000000000, 300.343000000000, 40.123000000000] [0.000000000000, 100.020000000000, -1.200000000000, 1.200000000000, 40.123000000000, 80.084000000000, 100.151100000000, 89.090800000000] [-1.200000000000, 65.126200000000, 100.020000000000, 40.123000000000, 1.200000000000] [40.1230, 0.0000, -1.2000, 100.0200, 40.1230] [0.0000, 82.0961, 100.0200, 31.1207, 21.1991, 6.1566, 40.1230, 100.0200, -1.2000, -1.2000] [] [1.2000, 28.0789, 40.1230, 17.0829, 300.3430, 40.1230, 50.1233, 66.1007, 100.0200, 0.0000] [100, 0, 70, 15, 16] [100, 12, 1, 0, 100, 40, 41, -1, 33, 100] [] [-1, 80, 32] ["", "--", "time why some here mean", "置业背书后一个启恒王东亮离谱安排一周小欣信号", "I'll or to be you're"] ["you're me would don't can how", "be", "?"] ["金立万元李经理电池开出来泡茶等着喜剧", "there so hey the she", "--", "配合", "-"] ["", "", "your"] ["", "one ok we his or we right", "开除参数表邮箱", "想不到", "", "伴我到学习位置三种哥们春节", "-", "right", "这首", "河南郑州"] ["s", "for yeah", "-", "now my", "-"] ["-", "on had I'll", "-"] ["", "something in just had time", "网景世家王子不好", "--", "", "-", "?", "雪城用眼香港聚兴采用游民下一部分刘亮盘磁带", "a", "派人"] ["f", "v", "z", "q", "r", "x", "b", "a", "f", "n"] \N ["f", "k", "t"] ["h", "f", "g", "i", "v"] [2023-12-14, 2024-01-31, 2026-02-18, 2023-12-17, 2024-01-31, 2025-02-17, 2024-01-17, 2023-12-20] [] [2024-02-18, 2023-12-11, 9999-12-31, 2023-01-15, 2024-01-08, 2014-08-12, 2023-12-13, 2023-12-20, 2024-02-18, 2023-12-20] [2023-12-20, 2024-01-09, 2024-08-03, 2023-12-16, 2023-12-18, 2024-06-30, 2024-02-18, 9999-12-31, 2024-01-08, 2024-01-19] \N [2024-08-03 13:08:30, 2025-02-17 00:00:00, 2024-06-30 12:01:02, 2024-01-31 00:00:00, 2023-12-14 00:00:00] [] [2023-12-13 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00] [] [2025-06-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-09 00:00:00] [2014-08-12 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00] [2023-12-15 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2002-08-08 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-01-15 08:32:59.123000] [2023-12-18 00:00:00, 2004-05-15 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00] \N [2024-02-18 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00] [2023-12-17 00:00:00, 2026-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00] +109 1867588248 251305842 0 -1858130809 2023-12-19 9999-12-31 2027-01-09 2024-02-18 \N well mean because how didn't some a - 谁那里 金成 it a like \N 0 43.116100 74.161100 300.343000 100.020000 9999-12-31 23:59:59 2023-12-17 00:00:00 2023-12-10 00:00:00 2024-08-03 13:08:30 2023-12-20 00:00:00 2024-01-17 00:00:00 2024-01-08 00:00:00 2008-06-01 00:00:00 [] [0, 1, 0, 1, 0, 0, 1, 0, 1, 1] [1, 0, 0] [0, 1, 0] [-128, -1, 82, -112, -1, -1, -124, 1, 1, 2] \N [0, 3, 1, 6, -128, -92, -1, -37, 7, 2] [127, 127, 0, 18, 127, -128, -128, -2] [1, 32767, 12062, 32767, -32768] [-29872, 1, 1, -32429, 245] [-2214, 1, -24452, 6471, -10691] [-5046, 0, 20622] [147483648, 32679, 1499283958, -257399768, 2147483647, 1081839024, 1643553892, -985813922, 147483648, -1] [32679, -1288831269, 1177414119, -215498423, 147483648] [-713442108, 147483648, -1984645713] [1, 0, 775339223] [147483648, 245, 1701490466717036388] \N [1701009250048291000, -5919160606598822006, 147483648, 1243568416, 147483648, 2083221680578058300, -3732249587359475968, -9223372036854775808] [-1, -1443878953613078274, 1, -80225771, 9223372036854775807, -9223372036854775808, 1, 6361894135835751480] [-1203546981, -2, 0, -1, 0, 9, 0, 0] [-2, 0, -1, 1, 1] [] [-1, -2, 1, 1, 0, -1, 0, -1112110938, 7, 3] [300.343000000000, 40.123000000000, 100.020000000000, 0.000000000000, 0.000000000000, 50.118600000000, 1.200000000000, 300.343000000000] [300.343000000000, 100.020000000000, 0.000000000000, 10.160300000000, 300.343000000000, 100.020000000000, 89.123800000000, 44.086500000000, -1.200000000000, 52.116500000000] [] [3.179600000000, 1.186600000000, 1.200000000000, 19.162800000000, 300.343000000000, 300.343000000000, 60.134500000000, 100.018500000000] [300.3430, 40.1230, 40.1230, 0.0000, 40.1230, 32.0713, -1.2000, 300.3430] [1.2000, 40.1230, 46.0927, 96.1869, 40.1230, 32.1942, 300.3430, 100.0200] [1.2000, 100.0200, 0.0000] [] [-1, 20, 96] [40, 0, 40, 15, -1] [300, 300, 100] [95, 0, 68] ["相恋回答问题新日射", "心理学", "分开", "you're can't here didn't really here did or hey to", "", "in", "歘俩注册表", "一起", "a", "I'll at your ok it's with that's don't"] ["her i of now", "-", "in that then have hey him your do", "c", "this"] [] ["when don't", "保险玲玲信号高级总代理正常", "平志伟付出了很错传真语文你是", "倪玲梅口语", "c"] ["so", "几台", "okay here could really", "新世纪日月丰", "w"] ["", "the it's it's he your then would", "he mean she your can't my", "--", "if okay want that not now what"] ["tell", "do", "-", "t", "how", "k", "-", "刘畅英莲惊喜金牌"] ["yeah hey that's I'm back had you're this", "you me is how me you're really are", "or", "-", "浪漫", "广场开出来有带你拓展零捌一些", "弱阳性", "v", "--", "her it's with it's from"] ["y", "h", "e", "r", "d", "x", "z", "g"] ["g", "t", "l", "y", "g"] ["j", "n", "b", "s", "h", "a", "t", "r"] ["h", "j", "r", "m", "t", "f", "i", "p", "n", "x"] [2026-01-18, 2027-01-16, 2023-12-20] [2024-01-17, 2024-02-18, 2026-02-18, 2023-12-13, 2023-12-13, 2025-02-17, 2024-07-01, 2024-08-03, 2024-01-19, 9999-12-31] [] [2024-01-09, 2023-12-17, 2024-06-30, 2024-01-08, 2026-01-18] [2023-12-19 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00] [2024-01-09 00:00:00, 2024-01-17 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2008-02-09 00:00:00] [9999-12-31 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00] [2026-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 9999-12-31 23:59:59] [2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [] [2023-12-20 00:00:00, 2023-12-13 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00] [2023-12-12 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2023-12-14 00:00:00, 2027-01-16 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00] [] [] +110 \N -2147483648 -1926721856 -1 2024-01-31 2023-12-12 2023-12-15 2023-12-20 it about could so her ? what 科技孟磊端口帮帮考前 水晶早上 一款 on I'm and was like 赵经理 0 1 27.051100 0.000000 40.123000 9.189700 2025-02-18 00:00:00 2023-12-16 00:00:00 2023-12-16 00:00:00 2023-12-10 00:00:00 2024-07-01 00:00:00 2024-07-01 00:00:00 2023-12-17 00:00:00 9999-12-31 23:59:59 [0, 1, 1, 1, 0] [1, 0, 0, 1, 0, 0, 1, 1, 1, 0] [] [1, 1, 1, 1, 0, 1, 0, 0, 0, 0] [90, 19, -32, 0, -28] [1, 127, 127, 1, 116] [1, 1, 36, 8, 1, -128, 127, 1, 72, 79] [] [-1, 32767, 245, -9575, -29877, -12972, 0, -32768, -11486, 0] [] [-7471, -17611, -32768, -277, 245] [245, -2223, -32768, 1, -31463, 28697, -19807, 9702, 245, 17688] [0, 2147483647, 1819722580] [] [1, 2109388461, -1, 0, 0] [1, -2021407256, 32679] [-9223372036854775808, 5680515750399859902, 6719113277516772800, -9223372036854775808, 245] [] [3093764401842274751, 1, -8442105055668899810, -9091104296258884690, 724633192102084620, -9223372036854775808, 830370456, -1131730454941289761] [-6605040, 9223372036854775807, 9223372036854775807, -4228747, 1, -1961360982, -7226470855365449747, -2168973, -3406331513456976174, -9223372036854775808] [0, -2, 1, -2, -1] [-1, 2, -1, -2, 3, 1, 0, 0] [7, -2, -1, 2, -1] [-1878187447, 0, -2, -1, -1, 4, -1, 0] [0.000000000000, 0.000000000000, 66.191400000000, 57.130600000000, 0.000000000000, 47.197600000000, 300.343000000000, 1.200000000000, -1.200000000000, 300.343000000000] [74.084200000000, 62.162800000000, -1.200000000000, 100.120700000000, 86.118700000000] [300.343000000000, 63.188700000000, 43.063400000000, 300.343000000000, 91.169200000000, 100.020000000000, 54.167000000000, 0.000000000000, 1.200000000000, -1.200000000000] [] [40.1230, 40.1230, 1.2000, 300.3430, -1.2000, 300.3430, 300.3430, 47.1560, 55.1258, 1.2000] [1.2000, 37.0098, 0.0000, 4.1571, 100.0200, 66.0473, -1.2000, 6.1826, -1.2000, 40.1230] [] [85.0679, 300.3430, 0.0000] [1, 0, 42, 31, 100] [62, 100, -1, -1, 300, 0, -1, 300, 300, 42] [] [] \N ["宣传", "--", "back", "back", "一声商人留一下创元听不懂全勤奖", "or", "不像", "up"] ["u", "智博惊喜金牌当兵", "when", "--", "to i can't her", "最新版本离职集成商换一家成绩小机器才回来", "新品百度最上面柯艳琦", "e"] ["time then this", "go mean this I'll they look you okay", "b"] ["签名", "", "明天会更好"] ["-", "回来今天", "姐姐", "全系列热插拔金达王平弯弯的月亮有没有前沿哥哥圈内最新版本", "慧眼照顾网卡小机器大键盘拿下有带你娃娃"] ["g", "going", "can"] ["just but been that oh as they hey good do", "你好吗心有", "合作差不多看来不爱公章代表美女"] ["k", "t", "s", "v", "q", "e", "g", "q"] ["p", "l", "x", "w", "f", "n", "f", "v", "r", "t"] ["s", "h", "v", "h", "g"] [] [9999-12-31, 2024-01-31, 2027-01-16, 2024-01-17, 2025-06-18] [2024-01-08, 2026-01-18, 2023-12-12, 2024-01-09, 2024-01-31] [2023-12-19, 2023-12-14, 2023-12-19, 2023-12-15, 2023-01-15, 2023-12-17, 2024-02-18, 2023-12-16, 2023-12-10, 9999-12-31] [] [] [2024-01-31 00:00:00, 9999-12-31 23:59:59, 2024-01-19 00:00:00, 2024-01-19 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00] [9999-12-31 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00] [2027-01-16 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30, 2023-12-14 00:00:00, 2023-12-17 00:00:00] [2027-01-09 00:00:00, 2025-02-17 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 9999-12-31 00:00:00] [2023-12-17 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123000, 2024-08-03 13:08:30] [] [2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00] [9999-12-31 23:59:59, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00] [2026-02-18 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00] [2024-07-01 00:00:00, 2026-01-18 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00, 2023-12-16 00:00:00] +111 936238756 592511108 313556233 1 2024-02-18 2024-01-08 2024-06-30 2026-01-18 do I'm she all h really tell ? then there my going with they just go how 创恒电子节日隆康但是节前供货 really get my so like we can can't from one yes 0 0 \N 43.005900 40.123000 -1.200000 2024-01-31 00:00:00 2023-12-19 00:00:00 2025-06-18 00:00:00 2023-12-20 00:00:00 2023-12-20 00:00:00 2014-08-12 00:00:00 2024-01-17 00:00:00 2024-07-01 00:00:00 [0, 1, 1, 1, 0, 1, 1, 1] [0, 0, 1, 1, 1, 1, 0, 0, 1, 1] [1, 1, 0] [1, 1, 1, 1, 1, 0, 0, 1] [-1, 26, 1] [0, 1, -1, -1, 1] [-66, 4, -1, -42, -74] [127, 127, -99, 106, 127] [-18538, -32768, 1] [5588, 1, 245, -14405, 19536, -19882, 0, -32546, 245, -28535] [] [] [-2147483648, -2147483648, -1342682904, -912907724, 1519284240, 0, 32679, -2147483648] [147483648, 1873015638, 232301272, -787730988, -2147483648, 32679, 2147483647, -1011364300, -1, -1] [-2147483648, 2147483647, 1948576949, -2147483648, -473755837] [] [7820449893031506511, 796894725893531401, 0, 1, 1885161752992023903] [-5027477405787548581, 147483648, 9223372036854775807, 7914921, -439854423464309312, 0, 9223372036854775807, 1773906079] [4251338710226764462, -9223372036854775808, -2307675191783732293, -1, 0, -9223372036854775808, 0, -1, -3718449965952414317, 4006315] [6325031594058694632, -7264693172493321578, 147483648, -1, 245, 2078335, -2324528810226256284, 6483338552961914370, 32679, -550818916] [7, -2103910313, -2, 1, -2, 4, 0, -1] [-2, 0, 1, -1, -2, 1, 1, -2, -2, -2] [-1, -1269833814, 1012073099] [0, 1, 7, -2, 1, -2, 1, -1] [54.054400000000, 300.343000000000, 48.182000000000, 57.051700000000, 39.025100000000] [99.061900000000, 86.076200000000, 93.063700000000, 96.192700000000, 80.101600000000, 1.200000000000, 300.343000000000, 1.200000000000, 300.343000000000, 0.000000000000] [62.171800000000, 96.057200000000, 300.343000000000, 74.102000000000, 1.200000000000, 23.018400000000, 20.096600000000, 100.020000000000, 51.021400000000, 61.041500000000] [1.200000000000, 28.196300000000, 1.200000000000] [0.0000, 100.0484, 86.1545, 300.3430, 0.0000] [61.0366, -1.2000, -1.2000, 15.1997, 300.3430, 68.0241, 100.0200, 15.1010] [] [100.0200, 77.0990, -1.2000, 40.1230, 0.0000, 61.0107, 40.1230, 1.2000] [100, 19, 0, 40, 94, 49, 0, 100, 8, 3] [100, 44, 53] [] [] ["切换", "肥肉", "--"] ["?", "?", "弄个百元整套多参与做私单经历电话", "the", "出现"] [] ["?", "?", "how", "联系一览表", "think"] ["", "s", "could not if me", "-", "--", "人家大棒汝南做系统进一步不太麻烦死金牌总代理发吧仔细", "--", "预计开公司中档核算", "节日", "检查"] ["两者", "-", "--", "--", "?", "well is are this when my mean at I'll some", "--", "she out when he been that's if really that's", "?", "-"] [] ["", "新年好", "were", "--", "for", "f", "didn't some something", "?", "when", "you my her right"] ["m", "y", "p", "h", "q", "s", "j", "h", "n", "m"] ["b", "b", "i"] ["k", "k", "f", "e", "l", "g", "v", "u"] [] [2025-02-17, 2023-12-13, 9999-12-31, 2006-10-11, 2026-01-18] \N [2023-12-18, 2023-01-15, 2026-02-18, 2024-07-01, 2023-12-16] [2007-02-18, 2023-12-16, 2023-01-15, 9999-12-31, 2024-06-30] [2004-03-05 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-09 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00] [] [2023-12-13 00:00:00, 2023-12-15 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2023-12-18 00:00:00] [2023-12-18 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00] [] [2023-12-19 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00] [2023-01-15 08:32:59.123123, 2023-12-13 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-08 00:00:00, 2025-06-18 00:00:00] [2023-12-14 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 2024-01-08 00:00:00] [2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00] [2024-07-01 00:00:00, 2023-12-11 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2026-01-18 00:00:00] +112 32679 147483648 -1 -1 2014-08-12 2023-12-13 2024-07-01 2027-01-09 特此证明货运打扰而不是 it's get -- she well who did now see ? 外面正事开头金城中心 泥石谁 \N 1 0.000000 17.165300 0.000000 -1.200000 2014-08-12 00:00:00 2023-12-15 00:00:00 2017-09-03 00:00:00 2024-01-09 00:00:00 \N 2024-01-31 00:00:00 2023-12-13 00:00:00 2023-12-16 00:00:00 [1, 0, 0, 1, 1] [] [0, 1, 0, 1, 0, 1, 0, 0, 0, 1] [1, 1, 0, 0, 1, 0, 0, 1] [127, 104, -128] [0, -106, -6, -128, -1, 9, -1, 1, 79, 127] [7, 103, 102] [110, -5, 0, -41, -49] [-22822, -11569, 19266, 7574, -32768, -4480, 245, -18259] \N [32528, 1, -32768, 1, -1, -16001, 1, -9744, -32768, 2054] [17252, -1, -27617] \N [32679, -2147483648, 1311885428, -2147483648, 1046061587] [32679, -306564766, -938227089, 697385504, 32679, 1853455871, 487095360, 0] [0, 1669448059, -924979697, -1604698220, 2147483647, 147483648, -1869360624, 1, 1, -2147483648] [0, -9223372036854775808, -1, 9223372036854775807, 9223372036854775807] [9223372036854775807, 37723171380704899, 2712152] [-1, 32679, 5421166, 9223372036854775807, 8043784716697388780, 3362754, 245, -876260568] [] [] [-1784371592, -1, -1, 0, 0, 1, -1691007824, -2, 2102876844, -1] [122726593, 1, -2, 4, -2, 0, 2030971035, -1636394504, 0, 0] [1, 0, -2] [100.020000000000, 1.200000000000, -1.200000000000, 0.000000000000, -1.200000000000, 300.343000000000, 40.123000000000, 28.009000000000, 93.100800000000, 23.184300000000] [52.175500000000, 6.002400000000, 1.200000000000] [-1.200000000000, 14.085000000000, 300.343000000000, 300.343000000000, 76.158500000000, 300.343000000000, 13.119300000000, 0.000000000000] [] [59.0537, -1.2000, 17.0973, 100.0200, 1.2000] [-1.2000, 72.0431, 18.0901, 300.3430, 100.0200, 2.1041, 0.0000, 16.1943, 0.0000, -1.2000] [-1.2000, 300.3430, 0.0000] [1.2000, 1.2000, 0.0000] [300, 100, 40, 0, 100, 40, 65, 300, 300, 0] [0, 40, 22, 0, 1, 9, -1, 6, 34, 28] [82, 10, 40, -1, 19, 53, 300, 300, -1, 0] [0, 93, 100, 40, 100] ["痛苦", "-", "do you know", "硅谷", "-", "y", "you're", "小灵通"] ["", "不忙", "优势不在哦一颗出团预算始终", "不完参数手册", "to"] ["赵洪应", "p", "all why about so the who will up had not", "五号", "--"] ["划破重命名新闻回忆光纤看完", "why", "相互互补明天会更好泡茶电脑打过", "代码", "time", "it's and did go", "无所谓试过万兴冯向建新瑞达计划外地", "--"] ["there mean how something did his who", "?", "q"] ["-", "相约", "阶段中小欣慰交流群贸易通圆圆", "", "would", "--", "then there really back", "I'll one", "刘广切换放假做不到扩充才做", "then that's of really didn't when some"] ["?", "进来", "like", "-", "", "-", "正品有缘推磨", "?", "查杀", "we have look can so how he don't"] ["填写", "-", "-"] ["g", "y", "u", "g", "v", "b", "z", "r", "s", "m"] \N ["t", "j", "a", "e", "y", "z", "s", "j"] [] [2023-12-20, 2008-01-23, 2024-06-30] [2024-01-31, 2026-02-18, 2023-12-15, 2024-01-08, 2023-12-11, 2024-07-01, 2024-08-03, 2023-12-19, 2006-07-14, 2025-06-18] [2024-01-08, 2008-08-07, 2023-12-15] [2023-12-13, 2023-12-18, 2023-12-16, 2024-02-18, 2023-12-12, 2024-02-18, 2023-12-13, 2024-07-01, 2024-01-17, 2023-12-20] [2014-08-12 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00] [2023-12-16 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00] [2023-12-12 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2023-12-17 00:00:00] \N [2025-06-18 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00] [] \N [] [9999-12-31 23:59:59, 2023-12-09 00:00:00, 2025-06-18 00:00:00] [2026-02-18 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-17 00:00:00] +113 638699174 658220826 -1 -1936697585 2026-02-18 2024-02-18 2024-02-18 2026-02-18 - 人见妹妹检查被授予这辈子手册 \N 考前 look he mean could look your or i for do e g 0 0 -1.200000 \N 24.180600 72.099500 2024-01-09 00:00:00 2027-01-09 00:00:00 2024-01-31 00:00:00 2025-02-18 00:00:00 2024-01-31 00:00:00 2023-12-18 00:00:00 2025-06-18 00:00:00 2023-12-15 00:00:00 [1, 1, 1, 1, 0, 1, 1, 1] [0, 0, 0, 1, 1, 1, 0, 1, 1, 1] [] [1, 1, 0, 0, 0, 1, 0, 0, 0, 0] \N [-33, 0, -128, -84, 1, 5, -1, 109, 4, 127] [2, -4, -9, 123, 9] [-1, -49, -48, 2, 6] [2014, 5406, -32768, 245, 32767] [] [32767, 0, 10446, -13241, 9171, 1, 23223, 245, 32459, -13461] [1, 245, -1] \N [-1, 32679, 32679, -1, 1997346545, -2147483648, 32679, -1192398309, 1, -1] [] [1, -2019180391, 2147483647, 1388834907, 147483648] [-1, 5093259311706306327, 565918149, 0, -1, 3691891088826694438, -9223372036854775808, 209525] [-7769590958028752650, 0, 972743, 147483648, -1, -1, 2167653283966474824, 147483648] [9223372036854775807, -4183899997082046771, -1, -9223372036854775808, 6566390487050271641, 0, 0, -258860550] [32679, -321413134, 0, 1, -3707622, 147483648, -9223372036854775808, 0] [0, 6, -1, 0, 0, 1, 1, -2, 1, -2] [0, -1, 5, 1, 5, 1, 1851532466, -535345410, -2, 1] [-2, 0, -1803885611, -1, 7, -151989213, 1, 0, -588774918, 2] [-1, -1039841505, 8, 0, -125830434, 0, 0, 0] [36.059800000000, 0.000000000000, 1.200000000000, 1.200000000000, 40.123000000000] [300.343000000000, 100.020000000000, 36.112500000000, 100.020000000000, 100.020000000000] [] [1.200000000000, 100.020000000000, 72.107300000000, -1.200000000000, 70.027600000000, 100.020000000000, 1.200000000000, 100.020000000000, 37.197100000000, 1.200000000000] [36.1876, -1.2000, 34.1830] [] [-1.2000, 40.1230, 35.1719, 24.0034, -1.2000] [96.1028, 0.0000, 0.0000, -1.2000, 76.0666, 17.0080, 14.0358, 97.0609] [] [300, 14, 48, 56, 100, 40, 100, -1, 1, 0] [] [40, 26, 100, 40, 300, 41, 100, 100, 44, 0] ["--", "--", "-", "周边劲舞团当地参数手册生活中打工的钻石路上挑战", "在欧普首先教练如果最起码便是清楚夏娜", "航道王之加入问道万普剪掉这辈子", "?", "k", "", "at"] \N [] ["?", "初八作风规格韩鸽飞组装", "right about at it like with could from the were", "?", "from", "--", "神仙", "选择便宜货运产生各个脱机张姐"] ["mean", "好好哦啊汇宝出来子夜方面还有", "安排", "跑跑卡丁车", "q"] ["--", "mean", "便是", "s", "as", "几下红雷就爱个", "", "as"] ["", "旅游", "of", "it's", "do as it up"] ["", "come can got think got look", "颜色", "just", "u", "-", "", "-"] ["v", "o", "w", "u", "d", "c", "w", "y", "w", "g"] ["v", "f", "a", "h", "o", "p", "n", "c", "o", "k"] ["e", "h", "i", "l", "x", "f", "z", "q", "y", "l"] ["r", "b", "s", "q", "o", "l", "y", "l"] [2023-12-19, 2010-08-19, 2023-12-12] [2024-02-18, 2024-01-17, 2024-02-18] [2025-06-18, 2023-12-20, 2023-12-13, 2023-12-15, 2023-12-12, 2023-12-14, 2023-12-12, 2024-02-18] [2024-01-09, 2023-12-12, 2023-12-18, 2023-12-17, 2024-01-09, 2012-09-20, 2026-02-18, 2023-01-15] [] [] [2023-12-16 00:00:00, 2023-12-12 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [9999-12-31 23:59:59, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 2023-12-17 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-18 00:00:00, 2023-12-19 00:00:00] [2024-01-19 00:00:00, 2023-12-16 00:00:00, 2024-08-03 13:08:30] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2024-01-09 00:00:00] [2023-12-16 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00] [2024-08-03 13:08:30, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-18 00:00:00, 2023-12-19 00:00:00, 2024-01-19 00:00:00, 2014-08-12 00:00:00] [] [2014-08-12 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-16 00:00:00, 2023-12-16 00:00:00] [2027-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00] [2023-12-12 00:00:00, 2024-01-17 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00] +114 \N 2147483647 -1949124156 147483648 2027-01-09 2024-08-03 2025-02-18 2024-01-31 my can't here with then were do or me up it when say you do 价格 - were can -- she what when this then the look something they \N 1 \N 0.000000 1.200000 1.200000 2023-12-19 00:00:00 2023-12-17 00:00:00 2024-01-31 00:00:00 2024-01-09 00:00:00 2023-12-15 00:00:00 9999-12-31 23:59:59 2024-07-01 00:00:00 2024-06-30 12:01:02.123000 [0, 1, 1, 1, 0, 1, 0, 0, 1, 1] [1, 0, 0, 0, 1] [0, 0, 1, 0, 0, 0, 0, 0, 0, 1] [] [8, -1, 122, 0, 127, -1, 0, 0] [1, 42, 0, 3, -48] [-123, 127, 0] [-128, 1, 4, -1, 5, 120, -128, -1] [-25190, -29478, 26919] [-20194, -27482, 26286, 26729, 18274] [-22694, -1, -2246, 0, 245, 1, -4759, 21739] [21239, 1, 1, 12000, 245] [] [32679, 32679, -1, 1, -1983664419, 0, 1720198342, 2147483647] [] [147483648, -1888376172, 882401362, -2147483648, 519468870, 1, 1363226856, -2147483648, 158429669, 32679] [8139347482655357761, -1, 32679, 0, 147483648, 9223372036854775807, -3259966874274542080, 2958183605561311993, 9223372036854775807, 9223372036854775807] [147483648, 5489770, 245] [1, 8973475444639611963, 7174861234817812852, -6839640326291792738, 32679] [8046840352802042458, 0, -9223372036854775808, -1, 0, -2425509434093670065, -7727330521111052111, 9223372036854775807] [3, -2, 0] [0, -1, 117861033] [1, -2, 818243553] [1, 1, 1] [100.020000000000, 3.166000000000, 40.123000000000, 39.109300000000, -1.200000000000] [300.343000000000, 300.343000000000, 10.166700000000, -1.200000000000, 300.343000000000] [0.000000000000, 82.047700000000, 1.200000000000] [] \N [-1.2000, 300.3430, 53.1961, -1.2000, 1.2000, 300.3430, 56.1815, 84.0051, 0.0000, 1.2000] [1.0106, -1.2000, 64.0374] [] [1, 50, 100] \N [300, 0, 98, -1, 100] [] ["get she", "", "p", "o", "I'll", "could out", "不止", "拿下"] \N [] ["didn't some I'll know a in", "?", "", "are his is what didn't ok", "what he's is will didn't tell one", "?", "no his up I'll it's been in a to i", "--", "?", "半天班多条余下"] ["-", "", "-", "x", "-", "--", "感悟不爱护哦整合首先股份五千一会见你好吗", "把控"] ["r", "k", "--", "was", "不走", "琵琶语", "最上面", "青青", "g", "--"] [] ["你早几天回来吧", "太远相处招待劲舞团详谈远方", "单子", "--", "why there yes his"] \N ["l", "h", "o", "v", "o"] ["a", "t", "r", "w", "t", "i", "p", "g"] [] [2023-12-14, 2023-12-20, 2023-12-18, 2024-01-09, 2025-02-18, 2023-12-11, 2023-12-10, 2024-02-18] [2023-12-09, 2023-12-10, 2025-02-17] [2024-01-17, 2024-01-09, 2024-02-18, 9999-12-31, 2023-12-14] [2023-12-09, 2023-01-15, 2023-12-18] [2023-01-15 08:32:59, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00, 2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-08-03 13:08:30] [2023-12-12 00:00:00, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00] [2024-01-17 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [2000-11-16 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00] [2024-02-18 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00] [2025-06-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00] [2024-08-03 13:08:30, 9999-12-31 23:59:59, 2026-02-18 00:00:00] [2023-12-19 00:00:00, 2024-07-01 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2011-03-13 00:00:00] \N [2023-12-18 00:00:00, 2023-12-19 00:00:00, 2024-01-19 00:00:00, 2023-12-16 00:00:00, 2005-12-26 00:00:00] [2023-01-15 08:32:59.123123, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00] [] +115 1661019687 -1487864942 -702886626 -2147483648 2025-06-18 2026-02-18 2023-12-11 2026-01-18 \N 只能 v out 尽管访问量六万宇瑞个人山下 限价沼气雅鹿沙呀库存效果多家认证均为 - 还就什么时间多少钱有什么聚会顾星力度四川 \N 0 \N -1.200000 11.079700 40.123000 2024-01-09 00:00:00 9999-12-31 00:00:00 2023-12-12 00:00:00 2023-12-16 00:00:00 2025-02-18 00:00:00 2025-02-17 00:00:00 2023-12-15 00:00:00 2024-06-30 12:01:02.123000 [0, 0, 1, 1, 0] [0, 1, 0, 1, 0, 0, 0, 1] [1, 1, 1, 1, 0, 0, 0, 1] [] [-26, 0, 0, -25, 0] [108, 2, -1, 107, -120] [1, 1, 0, 81, -128] [] [-32154, -20107, 0, 32767, -32768, 0, -17233, 32171, 0, 3473] [15675, -1, -28762, 32767, -32768] [14307, -30105, 1] [] [1782293784, -1673639408, 2147483647, -2084845973, -613420890] \N [-1585797277, -1971981833, -895787271, 0, 1941670728, 2147483647, 1479901423, -1] [] [-3528369189259489156, 5656106, -1, -4327173381337209855, 927633486830405243] [0, 147483648, 9223372036854775807, 245, 245, -1, 32679, -1398860495] [32679, 1, -2218435975869225734, 8778428076994214982, 32679] [147483648, -6130927, 3615118314557080227] [-1, 1, 31464544] \N [1404636647, -2, 0] [] [0.000000000000, 0.000000000000, -1.200000000000, 300.343000000000, 4.125800000000, 40.123000000000, 0.000000000000, -1.200000000000, 40.123000000000, 300.343000000000] [1.200000000000, 300.343000000000, 17.005500000000, 6.130000000000, 300.343000000000, 58.186700000000, 55.109000000000, 26.083100000000] [37.143400000000, 70.096500000000, 300.343000000000, 100.020000000000, 42.129800000000] [] [1.2000, 23.1340, 40.1230, 300.3430, -1.2000] \N [] [40.1230, 0.0000, 1.2000] [40, 12, 300] [300, 300, 1, 0, 300, 0, -1, 0, 100, 40] [100, 78, 100, 100, 72, 51, 1, 40] [] ["he well", "?", "?", "这话就是十二一直", "me", "", "金科华对方", "补丁", "记下", "up"] \N [] [] ["did do so if see the and how from just", "b", "come of", "深深", "what", "我到", "to then here when hey", "--"] ["that", "was been mean say been how all here because", "do he oh could when can well"] ["顺风跟着好哦啊屏保用友认证零点", "中心标志", "政策庙里大风而是实实在在", "yeah", "some some it as", "no", "李峰", "she get"] ["--", "安徽增票一次性", "will okay did didn't were didn't about I'm here", "", "can", "as", "how so we will", "邮件", "补上", ""] \N ["a", "v", "s"] ["j", "x", "y", "t", "u"] ["u", "d", "m", "r", "v"] [2023-12-13, 2025-02-18, 2024-02-18] [2023-12-09, 2023-12-10, 2024-01-09, 2024-01-31, 2010-12-09, 2023-12-20, 2025-06-18, 2005-12-27, 2024-02-18, 9999-12-31] [2024-01-08, 2023-12-12, 2024-01-17] [2024-01-09, 2023-12-10, 2009-01-09] [] [2014-08-12 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00] [2024-02-18 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00] [2023-12-11 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2024-06-30 12:01:02, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00] [2025-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-14 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00] \N [] [] \N [2027-01-09 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00] [2024-02-18 00:00:00, 2023-12-12 00:00:00, 2014-08-12 00:00:00] [2023-12-12 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2013-06-10 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-12 00:00:00] +117 -2147483648 2147483647 -2147483648 -2147483648 2014-08-12 2024-02-18 2025-02-18 2026-01-18 some she why at 不设置 第二款 something 解难行货对不对一条线优势 \N 1 42.141300 1.200000 48.150600 1.200000 2024-02-18 00:00:00 2024-01-17 00:00:00 2023-12-14 00:00:00 2023-12-16 00:00:00 2023-12-20 00:00:00 2024-08-03 13:08:30 2024-01-09 00:00:00 2023-01-15 08:32:59.123123 [1, 0, 0] [0, 0, 0, 0, 0, 1, 1, 1, 1, 0] [1, 1, 1] [] [2, -128, -50, 4, 1, 1, 0, -84] \N [-128, 3, -45, 0, 121, -1, -79, 1, 3, 9] [1, 1, -91, 3, 127] [-28345, 245, 4364, -15465, -572] [24959, 32236, -28736, 28466, 1, 1, -32768, 0, -30768, 21780] [1, -10323, 25204, 245, 6970, -1, 15527, 245, -6429, -9265] [] [-289506566, 32679, -1586041360, 2147483647, -2147483648] [-309988213, -1, 147483648, -127596546, -1152103261] [2078120099, 1, 147483648, 147483648, -1287217448, -814646306, -2147483648, -2147483648, -439783311, 460557282] [893399230, 1115366879, 846933128, 147483648, -1] \N \N [1, -1, 4687103] [-4638805987602244675, -1, 0, -1, -9223372036854775808, -9223372036854775808, -9134728407925489514, 1419068419, -1, 32679] [-1056988209, 1, 1, 1, -2, 1, -1, 1232896586, 5, 1231937806] [] [-2, -1, 0, -2, -1] [-2, -1, 0, -1541152646, 1, 1, -989628626, -353308565] [300.343000000000, 100.020000000000, 0.000000000000, 75.077700000000, 100.020000000000, 10.181500000000, 300.343000000000, 100.020000000000, 27.061900000000, 48.089800000000] [100.020000000000, 57.129500000000, 100.020000000000] [20.018700000000, -1.200000000000, 1.200000000000, 100.020000000000, 40.123000000000, 49.130600000000, 300.343000000000, 1.200000000000] [1.200000000000, -1.200000000000, 100.020000000000, 300.343000000000, 100.020000000000, 66.025800000000, 1.200000000000, 27.098400000000, 300.343000000000, 40.123000000000] [91.0963, -1.2000, 100.0200, 73.1390, 2.1401, 6.1685, 20.1429, 37.0417] [] [] [40.1230, 23.1527, 1.2000] [22, 0, 70, 300, -1, 1, 49, 0] [100, 0, 40, -1, -1, -1, 1, 48, 0, 93] [300, 0, 1, 100, 100] [] ["发邮箱", "-", "or to to but I'm your you say", "-", "群英", "h", "鸡巴", "that", "词典申请电影", "p"] ["i", "--", "发生过", "could", "j", "we", "打不出来", "?"] ["that's", "", "头痛"] ["信息部", "讲的", "u", "", "?", "because one yes I'll did are didn't we", "-", "come", "they at know me me", "卡巴斯基"] ["have", "?", "--"] ["电厂春江花月夜只因为汇祥效果百家周口废话接受", "五号", "didn't what but that's look of", "been", "of", "how my her have who and of", "猜到礼数几台太远小曹", "w"] [] ["请问这百年上有顺风公道话除非详情验货单同志们", "李峰", "go out for out she get", "?", "m", "", "?", "?"] [] ["v", "d", "p", "h", "e"] ["u", "x", "x", "d", "y", "g", "y", "m"] ["k", "n", "r", "w", "a"] [2023-12-17, 2024-08-03, 2024-06-30, 2023-01-15, 2024-07-01] [9999-12-31, 2024-02-18, 2024-01-19, 2023-12-09, 2024-01-08, 2024-08-03, 2024-06-30, 2024-01-17, 2024-01-19, 2027-01-09] [2024-01-09, 2023-12-12, 2023-12-15, 2023-12-16, 2025-02-17, 2025-06-18, 2026-01-18, 2024-07-01, 2024-02-18, 2023-12-11] [2023-12-20, 2025-06-18, 2024-01-17] [] [2023-01-15 08:32:59, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00] [2023-12-15 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2024-06-30 12:01:02] [2005-08-23 00:00:00, 2023-01-15 08:32:59, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00] [2023-12-20 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00] [2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2026-02-18 00:00:00] [2026-02-18 00:00:00, 2023-12-11 00:00:00, 2016-09-27 00:00:00] [2023-12-20 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02.123000] [] [2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00] [] [2024-01-19 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00] +118 -2147483648 32679 -1687560825 1866635518 9999-12-31 2023-12-13 9999-12-31 2025-06-18 - 常用哥们可以到赵燕燕平常新区机架服务器不及世纪 hey 库房小灵通瑞昌看不懂正确打架输得起登封听着 it ? 找你 -- \N 0 7.068100 1.200000 300.343000 40.123000 2025-02-18 00:00:00 9999-12-31 23:59:59 2025-06-18 00:00:00 2023-12-16 00:00:00 2025-02-17 00:00:00 2024-01-08 00:00:00 2023-01-15 08:32:59.123123 2024-01-08 00:00:00 [1, 1, 1, 1, 1] [] [0, 1, 0, 0, 0] [] [127, 21, 75, -1, 0, 117, -128, 127] [-65, -1, 0, -1, 11, 3, -1, 25] [] [-41, -128, 127, 127, 1, -1, 1, 1] [] [14240, -19504, -20769, 245, -11973] [0, -9210, -1] [-29632, -17414, 32767] [-2076027729, 1, 147483648, 1, -1924657831] [0, -1, -59436976, 32679, 0, 32679, -157811047, -1265376333, -2147483648, 32679] [147483648, 553087193, -1125328237] [] [-1, 147483648, 9223372036854775807, -1, -9223372036854775808, 32679, 0, 1, 993095324, -1] [-9223372036854775808, 1, 9223372036854775807, 4003543453886916120, 1, 4218449134551799625, 1, -9223372036854775808] [349952213790730503, 9223372036854775807, -1, 6951737, -1, -3600354685331517762, 32679, 4548136] [-1, -9223372036854775808, -9223372036854775808, -3485941300534287203, 32679, -9223372036854775808, 0, -1, -4338528731519133043, 1] [-943811163, 0, -2, 4, -2, -2, -2, -1, -1, 1083864722] [-1, -355975349, -1, 0, -2, 1, 9, 1] [-1506734058, 0, 0, -2, 0, -939736917, 0, 1, 0, 0] [-701235646, -2, 4, 1, -1, 0, 0, 4] \N [1.200000000000, 6.165200000000, 65.048700000000, 1.200000000000, -1.200000000000, 100.020000000000, 300.343000000000, 89.178300000000] [26.026700000000, 100.020000000000, 40.123000000000, 61.187700000000, 50.174600000000, 40.123000000000, 38.139400000000, 0.000000000000] [] [300.3430, 99.0972, 40.1230, 1.2000, 0.0000, 40.1230, 27.0872, 1.2000] [100.0200, 86.1344, -1.2000, 40.1230, 77.1117, 100.0200, 40.1230, 100.0200, 0.0000, 100.0200] [100.0200, 300.3430, 41.0430, 0.0000, -1.2000, 72.0997, 1.2000, 300.3430, -1.2000, 100.0200] [100.0200, 1.2000, 19.1572, 11.1667, 58.0762, 300.3430, 2.0739, 100.0200, -1.2000, 70.0146] [40, 0, 100, 300, 9] [38, 300, 92, 40, 100, 300, 62, 1, 0, 0] [52, 32, 78] [-1, 40, 300, 1, 40, 0, 40, 100, 36, 0] ["小灵通", "?", "", "why not was", "--", "can't could like", "-", "r"] ["和兴鼠标英文生活中", "yeah", "-", "天宇高级请重试姓江展开大会推荐周期台式机新联惠", "-", "模拟", "--", ""] ["my", "神偷", "do your all did not okay", "-", "this it's in", "what he her", "", "?"] ["this", "were because I'm they that's", "didn't"] \N ["I'll were yeah tell can't will okay", "l", "", "say not no me were she okay be it's", "一键还原对方", "that's", "go", "v"] [] ["i", "just", "", "拿下", "追求", "going", "his", "刻骨铭心转账强暴笔记本意义上次看到之间周期"] ["e", "r", "t", "j", "q"] ["c", "w", "s"] ["x", "q", "b"] ["h", "w", "w"] [2024-01-17, 9999-12-31, 2025-06-18, 2024-08-03, 2023-12-09, 2023-12-15, 2024-01-31, 2025-06-18, 2024-07-01, 2026-01-18] [2027-01-16, 2024-01-17, 2025-02-17, 2023-12-14, 2023-12-18] [9999-12-31, 2027-01-09, 2025-06-18] [2024-02-18, 2024-01-09, 2024-07-01] [2023-12-14 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00, 2024-07-01 00:00:00, 2014-08-12 00:00:00] [2027-01-09 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00] [] [] [2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [9999-12-31 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00] [] [2027-01-16 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00] [2023-12-16 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00] [2023-12-12 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00] [2025-06-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00] +119 147483648 985398925 0 0 2024-01-17 2023-12-10 2024-08-03 2024-01-08 ? me 郑州 was -- about know could think here about your can your out oh \N 1 1.200000 300.343000 97.168600 8.136800 2024-02-18 00:00:00 2023-12-16 00:00:00 9999-12-31 23:59:59 2024-01-31 00:00:00 2023-12-17 00:00:00 2024-01-08 00:00:00 2025-06-18 00:00:00 2026-02-18 00:00:00 [0, 1, 1] \N [0, 0, 1] [] [] \N [0, -1, -1, -51, 1, -1, -128, 127] [112, -128, -1, 1, 0, 127, -128, 127] [32767, -32688, -32768, 245, -14490, -1, 1, -22313] [16699, 1, 18747, -14911, 32767] [-32768, 9870, -21758, 32767, -5079] [9347, 32767, 27032] [-2147483648, -788666770, 2099827167, 1, -1, 1, 1556908745, 1678327693] [1003810382, -921917115, 32679] [] [1111530083, 308840739, 147483648] [-1, 1584815342, 245, -1, -1, 0, -7936951468294569910, -9223372036854775808, 245, -7909548012974159981] [245, -1, 32679, -9223372036854775808, -2430327032921804218] [-1, -1, -1] [147483648, 147483648, 245, 32679, -4308953158250975298, -9223372036854775808, 0, 506373576229437721, -9223372036854775808, 0] [-2, 6, 2026823322, 1, 0, -1, 1, -2] [0, 0, 0] [] [-2, 0, -1, 1, -2, -1, 1, 2] [1.200000000000, 300.343000000000, 300.343000000000] \N [300.343000000000, 300.343000000000, 100.020000000000, 1.200000000000, 100.020000000000, 300.343000000000, 300.343000000000, 0.000000000000, 0.000000000000, 69.000000000000] [76.115700000000, 300.343000000000, 93.077500000000] [81.0092, 0.0000, 29.1689, 100.0200, 0.0000] [50.1757, 1.2000, 31.0569, 1.2000, 64.0221] [] [14.1017, 100.0200, 48.0696, 83.0900, 40.1230, 0.0000, 100.0200, -1.2000, 300.3430, 0.0000] [100, 0, -1, 93, 100, 79, 1, 40] [100, 38, 0, 0, 38, 34, 23, 66, 74, 20] [] [] ["留个直接费用这中间开发", "her go", "from say here just", "", "or", "开发区不在新乡满意品放你杀时间谢晓娟连续祝你好", "-", "up why all well"] ["u", "", "-", "可选", "?", "未税", "when what out be be that this a", ""] [] ["got", "微软", "j", "something because okay oh what", "不太", "--", "g", "包含赶快客户类型林慧目前高杰又给"] ["帐户融入开公司很低", "所谓", "?", "格瑞办款", "我就短句操作输入选择深夜哈哈那边"] ["", "?", "?", "that that", "-", "say", "月内享受调货", "is"] ["早晨", "would you that's or right really", "-", "行货服务器", "?"] [] ["v", "m", "k", "e", "a", "h", "g", "s"] \N ["i", "s", "p"] ["e", "p", "o", "i", "q", "k", "q", "y", "y", "g"] [2027-01-16, 2024-02-18, 2023-01-15, 2023-12-20, 2013-02-19, 2024-01-19, 2024-01-19, 2023-12-18] [2023-12-15, 2027-01-09, 2023-12-14, 2025-02-18, 2023-12-17] [2023-12-20, 2026-01-18, 2024-01-09] [] [2023-12-19 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00] [2023-12-19 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00] [2024-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2023-12-17 00:00:00] [] [9999-12-31 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00] [2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 9999-12-31 23:59:59] [2027-01-09 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00] [2024-07-01 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00] \N [] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00] +120 -551616779 1307867559 2147483647 1146274782 2024-01-09 2027-01-09 2024-02-18 2023-12-09 in here all from been out been mean going 电脑上贰台这么做安排新拓大家放风筝影子南三环研修网 you're something 起床 ? 价值伤心一次性出面农行款且 ? 0 0 -1.200000 100.020000 300.343000 98.077100 2024-01-19 00:00:00 2024-01-17 00:00:00 2024-01-09 00:00:00 2027-01-16 00:00:00 2023-12-09 00:00:00 2024-01-19 00:00:00 2023-12-14 00:00:00 2023-12-16 00:00:00 [0, 1, 0, 1, 0, 1, 0, 0, 1, 1] [1, 1, 1, 0, 1, 1, 1, 1] [0, 1, 1, 1, 1, 1, 1, 1, 0, 1] [0, 1, 0, 1, 0] [0, 7, 126, -128, -15] [] [4, 0, 0, -128, 0, -36, 1, 8, 49, -1] [-1, 8, 0, -128, -4, 0, 0, 109] [10449, -25390, 30678] [-1, 1390, 30044] [] [-17166, -32768, 0, -29245, -3916, -28657, 32767, 32767] [-1, 2147483647, -1] [1, -632139265, 333069473] [2147483647, 314350763, 355629073, 754904752, -213016516] [1, -2006941281, 1910754661, 147483648, 32679] [-7893819644894042838, -1, 147483648, 245, 245] \N [] [-9223372036854775808, -6558050861186150113, 1] [] [0, -1, -1057860419, 0, -2, -1, -1, 0] [-1, 9, 1] [197782095, -2, -2] [] [] [-1.200000000000, 13.001400000000, 72.174100000000, 100.020000000000, 100.020000000000, 40.123000000000, 0.000000000000, 100.020000000000, 300.343000000000, 0.000000000000] [] [40.1230, 98.0663, 1.2000] [] [] [] [100, 63, -1, 100, 33, 62, 1, 29] [40, 1, 0] [0, 40, 1, 81, 0] [40, -1, -1, 87, -1, 300, 0, 0] \N \N ["--", "up", "g", "he it's we about but", "can he something oh ok you're you", "for from one", "?", "工程", "?", "?"] ["办完沧田联动盈盈成熟", "侯玉", "光缆独显那就好预计委会刘亚单单方法这点网景", "", "--"] ["q", "with was a a who are can't", "-", "--", "going", "i", "you're as just", "?", "see good there go were time not", "--"] \N [] ["", "", "--", "到来", "because", "", "that's", "瑞科付出过零捌卢有磁带注明效果感情捕鲸船之前"] ["p", "t", "e", "q", "r", "c", "r", "u"] ["y", "z", "k"] ["e", "n", "s", "l", "h"] [] [] [2024-02-18, 2023-12-20, 2014-08-12, 2024-08-03, 2023-12-11, 2024-01-31, 2024-07-01, 2003-07-28] [] [2025-02-18, 2024-01-08, 2024-01-31, 2024-01-19, 2025-06-18, 9999-12-31, 2023-12-20, 2023-12-16, 2023-12-19, 2023-12-19] [2025-02-18 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00] [] [2024-08-03 13:08:30, 2023-12-10 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00] [2023-12-09 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [2023-12-17 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00] [2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2023-12-20 00:00:00, 9999-12-31 00:00:00, 2024-01-09 00:00:00, 2016-08-20 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00] [2024-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00] [2023-12-15 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00] [2023-12-12 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00] [2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00] [2024-08-03 13:08:30, 2024-01-08 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2023-12-13 00:00:00, 2024-01-08 00:00:00] [2024-01-31 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59.123123] +122 32679 147483648 -1 1 2023-12-12 2023-12-17 2024-06-30 2027-01-09 \N 王全得 - 接口虚妄核算开票着呢太原启用冲突传美好好哦啊 做梦查查 文字可惜售后姑娘做生意洗衣服之前上网维泽那份 ? s 0 0 0.000000 -1.200000 19.055800 40.056900 2023-12-14 00:00:00 9999-12-31 00:00:00 9999-12-31 23:59:59 2023-12-18 00:00:00 2023-12-16 00:00:00 2024-02-18 00:00:00 2024-07-01 00:00:00 2023-12-12 00:00:00 [1, 0, 0, 0, 0] [1, 0, 0, 1, 1, 1, 1, 0, 0, 1] [] [0, 0, 0, 0, 0, 0, 1, 1] [-128, 127, 127] [-1, 0, -1, -1, 99, -59, 8, 91, 127, -57] [127, 0, -1, -88, -128] [] [-8271, -32768, 32767, -32768, 1] [21688, 0, 0] [10606, -32041, 9245, -1, 0, 0, 1, 16663] [245, 23733, 245] [0, -394485954, -1940910561, 32679, -1589471018, -1675915655, 2147483647, -1] [-2147483648, 32679, -1, 32679, -1, -1, -909242716, -1130350429] [0, 1701128288, 775280422] [1, -2081994664, 1, -2147483648, -1865323051] [896798, 9223372036854775807, 6444056857515348059] [-9223372036854775808, 7189260758405418709, -6179147763764519457, 1, 0] [885018895, 1072911149761523139, -9223372036854775808, 132549236039569568, -3176377045611912396, -9223372036854775808, -9223372036854775808, 32679, 0, 245] [258493365, -254672708, -645970471, -7096007, 1, 9223372036854775807, 147483648, -1] [3, 1, -2, -1, -415106578] \N [2, 1, 2] [0, 1, 1358192413, 0, -1, -2, -2, 1, 7, 0] [-1.200000000000, 59.094400000000, 40.123000000000, 43.006400000000, 88.169200000000, 0.000000000000, 31.136500000000, 36.062300000000, 54.048100000000, 38.123000000000] [1.200000000000, -1.200000000000, 0.000000000000] [40.123000000000, 300.343000000000, 51.040700000000, 100.020000000000, 0.000000000000] [19.064800000000, 37.117600000000, 26.001500000000, 40.123000000000, 1.200000000000, 300.343000000000, 89.079200000000, 1.200000000000] [1.2000, 300.3430, 1.2000] [40.1230, 31.0264, 91.1083, 300.3430, 11.0532, 100.0200, -1.2000, 50.0909, 0.0000, 84.0050] [1.2000, -1.2000, 40.1230, 100.0200, 300.3430] [0.0000, 82.0059, 40.1230, 79.1962, 0.0000] [300, 0, 40, 100, 72, -1, 100, 1, 94, 39] [38, 300, 100] [92, 58, 300, 300, 100] [1, -1, 73, 69, 0, -1, 1, 1] \N \N [] ["what", "me", "o"] ["", "眼泪", "加载美丽电话中档", "no", "your"] ["个月还在刻骨能早点战友", "about", "this we for okay some think back"] [] ["some", "get", "乐凯"] ["p", "h", "m"] ["b", "a", "b", "o", "d", "j", "x", "i", "p", "h"] ["q", "i", "s", "p", "l"] ["v", "x", "u", "j", "k", "n", "e", "t"] [2025-06-18, 2024-02-18, 2023-12-11, 2027-01-16, 2027-01-16] \N [2014-08-12, 2026-02-18, 2024-06-30, 2026-01-18, 2024-01-17, 2023-12-15, 2024-02-18, 2027-01-16, 2023-12-18, 2025-06-18] [2024-01-09, 2024-02-18, 2025-02-18] [2027-01-16 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00] [2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00] [] [9999-12-31 23:59:59, 2026-01-18 00:00:00, 2027-01-09 00:00:00, 2005-09-14 00:00:00, 2024-02-18 00:00:00] \N [2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00, 2023-12-19 00:00:00] [] [2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2023-12-16 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00] [2023-12-09 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-17 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-16 00:00:00] [2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00] [2026-01-18 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-15 00:00:00, 2024-01-17 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [] +123 -1 -479267175 2147483647 2125396964 2024-06-30 9999-12-31 2027-01-09 2025-06-18 世安 五洲佩利动作汉威双核加入另外报价格花月夜节点 未来付出了总部众诚腾讯节省还有见面很大 something 等级五万能不鞥金信飞鸽东京开封 - - tell my me in have i ok did 1 0 100.020000 \N 300.343000 300.343000 2025-02-17 00:00:00 2023-12-18 00:00:00 2024-01-31 00:00:00 2025-06-18 00:00:00 2025-02-17 00:00:00 2024-02-18 00:00:00 2026-02-18 00:00:00 2024-02-18 00:00:00 [1, 1, 1, 0, 0, 0, 0, 0, 1, 1] \N [] [] [-128, 0, 1, 7, -63] [127, -100, 9, 127, -27] [-1, -1, 119] [-55, 2, -117, -1, 0, -16, 1, -62] [] [] [] [18500, -1, 61, 32767, 22687, 18307, -31388, 245] [147483648, 32679, -1, -2147483648, -755526159, 1, 1, -2147483648, -982039914, 147483648] [-541503957, 0, 2147483647] [-2147483648, 2079204630, 32679, -1, -329183827, 2147483647, 147483648, -696585171] [2129645923, -1978974675, 1, 32679, -2147483648, -919676737, 2147483647, 1431145029] [227840502166741942, 245, 6066336086566381053] [5522222549921178368, 2205499620074615085, 4354843689247773178, -1, 5705022, 1618120813498118468, 245, 9223372036854775807, -6475400, -2229477591770648208] [] [9223372036854775807, 4366686250890923467, 8378084830465222929, -7330013232223466468, -2635037236422169170] [] [-2, 1, 430167912, -2, -2] [1, 1353767182, -1, 1, 0, -1468378756, 3, -1, -848201079, 0] [-2, 1, -1, -1, 1] [1.200000000000, 1.200000000000, 34.090200000000] [-1.200000000000, 20.045800000000, -1.200000000000, 40.123000000000, 40.123000000000] [100.020000000000, 95.050300000000, 21.110100000000] [40.123000000000, 18.080200000000, 300.343000000000, 100.020000000000, 1.200000000000, 1.200000000000, 300.343000000000, 0.000000000000, -1.200000000000, 52.068000000000] \N [100.0200, 40.1230, 62.0839] [1.2000, 40.1230, 59.0316, 56.1719, 27.1701] [] [0, 9, 300] \N [] [0, 100, 56, 1, 300] ["him got I'm in well ok he's", "yes think i right but do his", "my", "--", "and here now were look", "便是没结束", "--", "have"] ["", "of really", "备份都说", "-", "好不好", "原则真人嵩县", "其余", "from"] ["didn't when don't was at was time but", "the", "I'm", "?", "亲戚", "-", "会话帮忙改改壹仟伍分别侯景瑞贺壹仟你还不去", "x", "图形好评海涛有谁焦小", "武警总队交货姚明"] ["what", "", "--", "-", "河南郑州扩展板不已整合浮动", "did his yeah a", "?", "say"] ["didn't didn't up on were", "", "?", "老了世上", "-"] ["-", "this", "can't really", "分公司", "j", "?", "not in and he then will been can not", ""] ["心有东京不论军港意识号码", "联系一览表", "x", "--", "疑问"] ["there", "?", "汇众天下", "--", "think"] ["m", "i", "u", "q", "i", "y", "x", "o"] ["n", "t", "g"] ["n", "e", "i", "z", "q"] ["r", "o", "j", "t", "m"] [2024-07-01, 2024-06-30, 2023-12-11] [2023-12-16, 2024-01-31, 2023-12-19, 2024-02-18, 2024-02-18] [2027-01-16, 2023-12-15, 2019-02-01, 2026-01-18, 2023-12-19, 2026-01-18, 2023-12-19, 2024-02-18, 2023-12-13, 2027-01-16] [2014-08-12, 2024-01-08, 2024-07-01, 2026-02-18, 2024-07-01, 9999-12-31, 2001-02-18, 2025-02-18, 2024-07-01, 2023-12-10] [2024-06-30 12:01:02, 2023-12-10 00:00:00, 2023-12-18 00:00:00] [2025-06-18 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] [] [2023-12-16 00:00:00, 2023-12-18 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2024-01-19 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-13 00:00:00, 2026-02-18 00:00:00] [2023-12-20 00:00:00, 2026-01-18 00:00:00, 2023-12-17 00:00:00] [2023-12-15 00:00:00, 2004-12-10 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00] [2027-01-16 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00] [2023-12-17 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00] [2024-01-08 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2014-11-24 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2024-01-19 00:00:00] [2024-01-19 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00] +124 2147483647 32679 0 32679 2024-02-18 2023-12-16 2023-12-10 2025-06-18 could -- of -- are some yeah out her there me 疲劳 of a then 盘磁带方式吃过饭彭巧语正常打架越来越来青青草 \N 1 0.000000 80.120300 -1.200000 100.020000 2023-12-16 00:00:00 2023-12-19 00:00:00 2027-01-09 00:00:00 9999-12-31 00:00:00 2023-12-11 00:00:00 \N 2023-12-17 00:00:00 2024-06-30 12:01:02.123000 [0, 1, 1, 0, 1] [1, 0, 1] [1, 0, 0, 0, 1] [0, 0, 1, 0, 1, 0, 1, 0, 0, 1] [-1, 127, 127, -128, -1, 33, -128, -128, 0, 90] [] [127, -68, -1] [0, -118, 5, -36, 0, -110, 1, 62] [-1, -1, -12477, -1, 0, -23049, -21387, -31638] [31685, -30958, -1, -10942, -19646, 623, 245, 31323] [-20692, 1, 23163, 21744, 1, -12580, 574, -30438] [] [-2147483648, 2147483647, -884124302, -232937315, -673589708] [] [] [-1640278646, 1075642951, 403962173, -1, 2048291192] [3139913, -2783145026056637492, 32679, 9223372036854775807, 9223372036854775807, 9223372036854775807, 245, 7670379312831548123, 5845805034366515852, -620141942] [-9223372036854775808, 147483648, -7745132283340129133, 147483648, 147483648, 0, 245, -9223372036854775808] [0, 9223372036854775807, 245, -1090080369671474235, 32679, 245, -1, 245, 9223372036854775807, -992671410] [-4945707, 147483648, -9223372036854775808] [-2, -2006712593, -2, 1, 1, 1, 0, 0] [] [4, 1, 0, 0, -2] [-1, 4, 1, 1, 1] \N [27.165900000000, 56.145100000000, 66.128500000000, 40.123000000000, 1.200000000000, 15.152500000000, 40.123000000000, 48.015700000000, -1.200000000000, 300.343000000000] [70.102700000000, -1.200000000000, 5.158900000000, 2.063900000000, 86.004000000000] [] [57.0988, 95.0848, 7.1124] [0.0000, 0.0803, 100.0200] [4.1284, 62.1317, 33.1406, 48.0778, 0.0000] [54.0975, 0.0000, 1.2000] [50, 40, 300, 78, 40, 1, 91, -1, 1, 9] [40, 1, 34, 32, 89] [300, 1, -1, 40, 1, 300, 14, 23, 300, 1] [0, 39, 97, 69, 300, -1, 40, 100, 100, 40] ["-", "-", "f"] ["would", "通过彩页哥哥仔细兴致圣荣电话给我帮帮", "go", "just", "好了", "I'm but how really", "吉林", "濮阳发到一舟跟单处理器拐卖看到熊朝华病毒"] ["hey don't", "x", "-", "", "t"] ["?", "资讯普惠清理下着", "充满图标都还累了兄弟标志", "乱七八糟摩擦喜剧", "can't hey now and did because you like that know", "with", "哟啊充满好多付款坐在沧田都有", "相恋", "?", "but look just come a yes was will"] ["陇海", "a a I'm what were were if yes to on", "上演文件夹汉威赞美诗联盛珊瑚开除", "-", "哪里"] ["事务所", "?", "-", "his want her and all do", "?", "not of you have don't they then did to with", "", "资金", "-", "看吧"] ["you oh that why don't out that I'm", "--", "my that why with", "--", "do"] ["好不好卡巴斯基", "were back from can then didn't", "?", "重命名", "--", "王佩丽", "here she of", "景霞", "那么大每一小时下雪看看分区", "they the get didn't are from if who was be"] ["p", "a", "z"] ["a", "m", "m", "q", "c", "e", "n", "p"] ["o", "d", "v", "o", "i", "a", "x", "z", "o", "q"] [] [9999-12-31, 2027-01-16, 2023-12-16, 2023-12-15, 2023-12-10, 2027-01-16, 9999-12-31, 2024-01-09, 2027-01-16, 2026-02-18] [9999-12-31, 2023-12-10, 2023-12-16] [2023-12-09, 2023-12-17, 2027-01-09, 2025-06-18, 2024-07-01, 2023-12-13, 2023-12-19, 2024-07-01, 2025-06-18, 2024-08-03] [9999-12-31, 2023-12-12, 2025-06-18, 2023-12-13, 2025-02-18] [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00] [2024-01-19 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00, 2023-12-16 00:00:00] [2024-01-31 00:00:00, 2023-12-12 00:00:00, 2026-01-18 00:00:00] [2024-08-03 13:08:30, 2024-08-03 13:08:30, 9999-12-31 00:00:00] [2023-12-13 00:00:00, 9999-12-31 00:00:00, 2024-01-19 00:00:00] [2026-02-18 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00] [2023-12-19 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 2024-08-03 13:08:30, 2025-02-18 00:00:00] [2026-02-18 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30] [] +125 1731039365 1247246672 2147483647 0 2025-02-18 2024-02-18 2024-02-18 2025-02-17 配置先付 been 合作 think his to will I'll just - u 也有 \N 1 300.343000 -1.200000 67.183600 -1.200000 2026-02-18 00:00:00 2023-12-12 00:00:00 2023-12-10 00:00:00 2027-01-16 00:00:00 2025-02-18 00:00:00 2024-01-19 00:00:00 2025-02-18 00:00:00 2025-02-17 00:00:00 [0, 0, 0, 0, 0, 1, 1, 0] [1, 0, 0, 0, 0] [1, 1, 0, 0, 0] [0, 1, 0, 0, 1] \N [-128, -111, -128, -128, -128, -51, 22, 1, -43, -128] [] [-128, -1, -128, -128, -62] [-16671, -19124, 1, 27831, 0, -19686, -6469, -18151, -1, -32768] [-22529, 22694, 9487] [1, 30648, 1, -1, -10615, 0, -31016, 32767, 1, -1] [245, 1, -32629, 19329, 22669] [0, 880354584, 1793402746, 1110541122, -458193659] [1428703651, -800633426, -1447565172, -1, 0] [0, 32679, 1, -2147483648, 147483648] [-1395262977, 564379333, -2147483648, 1795651618, 290911335] [-8199729198154659232, -1503052579, 245, 1, 245] [245, 32679, -483617917, -1, -1186976035] [0, 32679, 1, -1581362390722850638, 1] [9223372036854775807, 32679, 245] [7, 0, -1470716071, -1, 2110492662, -1109812272, 990842911, -1979488386] [] [0, 5, 767009339, -1, -2, -1, 1, 604958568, 1, 0] [-1, 0, 0, 9, 1363727272, -2, -1, -1, 6, 0] [-1.200000000000, -1.200000000000, 300.343000000000, 40.123000000000, 300.343000000000] [100.020000000000, 1.200000000000, -1.200000000000, 1.200000000000, 40.123000000000] [300.343000000000, 1.200000000000, 87.053600000000] [3.142900000000, 1.200000000000, 3.002900000000, 0.000000000000, 100.020000000000, -1.200000000000, 300.343000000000, 300.343000000000] [0.0000, 300.3430, 1.2000, 59.1288, 300.3430, 42.0562, 59.1114, 300.3430, 300.3430, 300.3430] [48.0225, 100.0200, 100.0200] [66.0025, 44.1922, 0.0000, 300.3430, -1.2000, 40.1230, 1.2000, 100.0200, 75.1453, 43.1555] [] [100, 0, 40] [100, -1, 1] [-1, 300, 100, 0, 17] [100, 48, 1, -1, 24, 19, 1, 89] ["杨宗中科开通曾经鸡巴马后宏运动手脚热插拔", "战友", "f", "太不", "?", "because are they a i", "your", "?"] [] [] [] \N ["", "", "i", "办完", "e", "up got are hey could", "would could his get", "at so that's think why she didn't it's"] ["授狗", "日益轩明产品真人生病赞美诗", "玉冰"] ["开出来正信", "--", "天孝拿货地矿研究所老师吴静图像", "look I'm you're", "顾星"] ["a", "m", "k", "x", "l", "v", "l", "l"] ["t", "w", "y", "q", "c", "t", "u", "q"] ["e", "c", "c", "r", "o", "s", "p", "y"] ["f", "y", "d"] [2007-08-17, 2024-01-09, 2024-07-01, 2025-02-18, 2014-08-12, 2023-12-14, 2025-06-18, 2025-02-17] [2014-08-12, 2027-01-09, 2024-02-18, 9999-12-31, 2023-12-16, 2023-12-17, 2023-12-14, 2023-12-16, 2027-01-16, 9999-12-31] [2023-12-13, 2024-02-18, 2027-01-09, 2014-08-12, 2024-01-17, 2023-12-14, 2024-06-30, 2024-01-09, 2024-02-18, 2024-08-03] [2023-12-16, 2023-12-10, 2024-02-18, 2023-12-15, 2025-06-18, 2024-02-18, 2023-12-15, 2023-12-14, 2023-12-18, 2025-02-18] [2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59, 2023-01-15 08:32:59, 2023-12-18 00:00:00] [2023-01-15 08:32:59, 2027-01-09 00:00:00, 2024-01-09 00:00:00] [2025-06-18 00:00:00, 2026-01-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00] [] [9999-12-31 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00] [] [] [2023-12-14 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00] \N [2023-12-12 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00] [2023-12-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-11 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00] [] +126 2147483647 1 -1025538764 328591582 9999-12-31 2024-02-18 2023-12-19 2024-01-08 \N ? 倒掉 项目这块 - he's why hey from of he's going at \N 0 -1.200000 16.118500 40.123000 40.123000 2023-12-10 00:00:00 2023-12-12 00:00:00 2023-12-16 00:00:00 2024-01-31 00:00:00 2027-01-09 00:00:00 2024-01-09 00:00:00 2023-12-13 00:00:00 2024-06-30 12:01:02.123000 [1, 0, 1, 1, 0, 1, 1, 1, 0, 1] [0, 0, 0, 0, 0, 1, 0, 1] [] [] [51, -128, -106, 127, 127, 64, 127, -128, 26, 1] \N [-128, 3, 9, -1, -128, -1, 127, 0, -1, -17] [] [245, 8658, -1] [245, 32767, 0] [13796, -22643, 32767, -32768, -3796, 4804, 245, 11693, 838, 7522] [28274, -1, 6725, 26167, 22468] [1, 384395329, 2147483647] \N [0, -55413683, 1] [610699126, 651067517, 233244297, 2147483647, 2147483647, 2147483647, 0, -1] [-3780478, 245, 669762258551120902, 9223372036854775807, 8155523, 0, 245, 245, 245, 147483648] [-9223372036854775808, -550160484, 9223372036854775807, -1169933, -1] [1, -9223372036854775808, 7423294073566476249, 245, 2855933, 147483648, 0, 32679] [1, 1715523242405843221, 147483648, 4072103417828276688, -4885684969233609725, -9223372036854775808, -1043086956, 0, 9223372036854775807, 32679] [-2, 0, 6, 1, 1] [] [-2, 1, 6] [1, 1, 0, -2, 794714867, 1, 1, 1, -2, -2] [] [0.000000000000, 59.168300000000, 88.037100000000, 31.131000000000, 83.120200000000] [100.020000000000, 100.020000000000, 46.143600000000, 1.200000000000, 95.185500000000] [] [] [26.1537, 1.2000, 100.0200, 40.1230, 40.1230, 56.1431, -1.2000, 93.0845] [100.0200, 100.0200, 0.0000, 300.3430, 40.1230] [81.1605, 1.2000, 300.3430, 53.0447, -1.2000] [40, 1, -1, 1, 0, -1, 300, 300, 0, 40] \N [9, 1, 51] [] ["?", "报名", "w", "i", "", "信用社春江导入", "", "how about at"] ["--", "请他", "q"] ["", "with", "why out", "电视", "回公司", "right", "黑帮", "", "something what so want well tell with oh going", "下一部分周哥轻轻不起"] ["that's up that a will was say just from go", "招工", "--", "--", "been", "-", "-", "--", "智慧", "白明伟法律这里前一个用户商务好卖长江路十来年"] [] ["?", "tell when", "here", "who", "did this and have right tell he's", "why because yes want can't right", "", "方向"] ["-", "", "j"] ["n", "嘉运达除非心理学公寓标底", "this think my yeah", "何时", "发发做成欣慰你们", "我同学结婚来庆祝大客户品放", "it's don't mean who don't his", "-"] ["g", "f", "q"] ["f", "x", "l", "u", "v", "w", "q", "n"] ["d", "q", "q", "s", "d"] [] [2023-12-18, 2025-02-18, 2024-06-30, 2024-02-18, 2011-01-10, 2027-01-16, 2023-12-19, 2023-12-17] \N [2024-06-30, 2024-07-01, 2024-08-03, 2025-06-18, 2024-02-18] [2024-08-03, 2023-12-16, 2024-01-09, 2013-07-26, 2023-12-11] [2023-12-09 00:00:00, 2015-06-01 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59, 2024-01-19 00:00:00, 2023-12-16 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00] [2023-12-15 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [] [2024-01-31 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00] [2023-12-16 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2016-12-08 00:00:00, 9999-12-31 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00] [2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00] [] [] [2023-12-13 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-17 00:00:00, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2024-01-08 00:00:00, 2027-01-09 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-11 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00] +128 147483648 32679 -1 869724582 9999-12-31 2025-02-18 2023-12-11 2023-12-19 me all ? 那次多星也许下单长期还得真理 something ? ? 0 1 48.062100 0.000000 77.061800 300.343000 2024-01-31 00:00:00 2025-06-18 00:00:00 2023-12-09 00:00:00 9999-12-31 00:00:00 2025-06-18 00:00:00 2024-01-19 00:00:00 2024-01-08 00:00:00 2026-02-18 00:00:00 [1, 1, 0, 0, 0, 1, 1, 1] [1, 1, 0, 1, 0, 1, 0, 1] [0, 0, 0, 1, 1] [0, 0, 1, 0, 1] [-128, 2, -1, 4, 0] [] [] [-122, -128, -128, -30, -128, -128, 127, -128] [-32768, 31595, 31861, 20894, 32628, 0, 1, 245] [-1, 0, -5020, 18578, -1] [] [] [1322674557, 0, -2147483648, 32679, 2147483647, 2147483647, 838186406, 1] [2147483647, -725260467, 1107416103, 2147483647, 1966896759, -357301372, -2147483648, 856240251] [1, 32679, -2007636469, 83775439, -2147483648, 32679, 1, 565105939] [1, 1, 735039484] [7976035294212359030, 1736460899, 1, -1, -8435290648452561897, 8266122633621658468, 7364276601030260175, -1] [32679, 245, -3492304973487810506] [-8943261806571686794, 1, 147483648, -1, 1365510856480088030] [-1680253, 1, 1, 32679, -9223372036854775808] [7, 0, 9, 2, -1, -2, 380160967, -2, -2, 1] [-2, 2, 0, -2, 5] [] [-1, 5, -1, -2, 1] [] [-1.200000000000, 300.343000000000, 40.123000000000, 39.059500000000, 2.058300000000, 0.000000000000, 0.000000000000, 0.000000000000, 3.071900000000, 40.123000000000] [] [300.343000000000, 300.343000000000, 100.020000000000, 300.343000000000, 100.020000000000] [32.1010, 66.0434, 81.1820, 300.3430, 0.0000, 73.0528, 0.0000, 65.0219] \N [0.0000, -1.2000, 1.1643] [300.3430, -1.2000, 100.0200] [1, 1, 25, 94, 40] [0, 0, 0, 100, 40] [] [30, -1, 1, 300, 1, 0, 34, 64, 49, 0] [] ["can't him", "虽然", "这几太难", "中信怎么找听产品年前要求数目明天其人自豪", "--"] [] [] ["", "m", "out", "did", "?", "it's", "--", "", "can't", "?"] ["沃尔普两者真实工分儿", "me as like okay her that would okay can", "r"] ["s", "-", "-", "?", "验货单"] ["申请", "not", "--", "-", "款且存储附件排列中病毒详细点存储"] ["k", "w", "c", "v", "v", "s", "y", "m", "e", "v"] \N ["f", "w", "k", "b", "r", "k", "u", "g", "f", "f"] ["b", "b", "g"] [2023-12-14, 2024-01-31, 2026-02-18, 2023-01-15, 2025-02-17, 2024-02-18, 2023-12-19, 2023-12-09] [2023-12-19, 2023-12-19, 2024-02-18, 2024-01-17, 2024-08-03, 2023-01-15, 2023-12-16, 2023-12-20, 2023-12-09, 2023-12-17] [2023-12-15, 2024-01-08, 2023-01-15] [] [2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00] [2023-12-09 00:00:00, 2023-12-18 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00] [2024-07-01 00:00:00, 2009-04-07 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-19 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-09 00:00:00] [] [2027-01-09 00:00:00, 2025-02-17 00:00:00, 2008-12-08 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123000, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00] [9999-12-31 00:00:00, 2023-12-17 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00] [2024-01-17 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00] [2023-12-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] \N \N [2026-01-18 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2023-12-09 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 2000-05-16 00:00:00] +130 \N 1137602462 2147483647 0 2024-07-01 \N 2025-02-18 2023-12-16 中龙 \N \N 自导关于适合才好到时新宇考勤星月 f o ? 1 0 100.020000 64.044700 57.132500 29.123100 2023-12-14 00:00:00 2027-01-16 00:00:00 2023-12-14 00:00:00 2024-01-09 00:00:00 2024-01-17 00:00:00 2023-12-14 00:00:00 2014-08-12 00:00:00 2023-12-18 00:00:00 [0, 1, 1] [0, 1, 1, 0, 1, 0, 0, 1, 1, 1] [1, 0, 1] [] [2, 52, -1, 1, 0, -55, -1, 1] [42, 54, 7, 2, -1] [0, 0, 7, -1, 5, 28, -128, 0] [] [48, -32768, -32768, 20312, -1] [245, -32768, -32768, 1, -1, -9801, -32768, -1, 21916, -2601] [5311, 0, -14856, -21985, -2698, 32767, 11096, 1] [11615, -32768, -32768, -32278, -1] [2147483647, -1, 147483648] [1948728763, 1626279905, -130697732] [-2147483648, -460356898, 147483648, -1, 147483648] [] [0, 32679, 0, -2091051327795862466, -9223372036854775808, -1, 147483648, -1] [245, 32679, 0, 32679, -2925627453877317651, -9223372036854775808, -6369549872493703658, 9223372036854775807] [3638966436086490618, -9223372036854775808, 32679, 32679, 1, -8699916941648742068, -6387182058476946981, 147483648, -1, 245] [-9223372036854775808, 9223372036854775807, 147483648, -1, 32679, -1369469800208076259, 32679, -1, 9223372036854775807, -9223372036854775808] [1, 7, -2, 0, -2] [-2, 8, -2, 0, -1, 559181050, -1, 1472818986] [-1, 4, 1, 0, 2] [1, -2, 3] [68.157000000000, 1.200000000000, 300.343000000000] \N [] [] [100.0200, 300.3430, 68.1028, 7.1209, 28.0380, -1.2000, 1.2000, 1.2000, 81.0429, 40.1230] \N [] [40.1230, 0.0000, 67.1394, -1.2000, 0.0000] [] [36, 1, 40, 40, 40, 30, 100, 0, 40, 100] [11, -1, 1, 40, 1, -1, 99, 300] [0, 100, 25, 33, 1, 100, 78, -1, 63, 300] \N [] ["he", "独立显卡自导乔总吉林请到回报长期分忧股份有限公司", "供应质保金很多", "", "y"] ["when", "one look had", "--", "瑞贺", "", "", "", "you", "been", "it"] [] ["-", "流程", "out"] [] ["--", "电话给我", ""] ["m", "b", "g"] ["l", "m", "z", "q", "s"] ["g", "j", "d", "k", "d", "i", "h", "i", "w", "a"] ["b", "m", "v", "m", "g", "r", "i", "g", "q", "n"] [2023-12-17, 9999-12-31, 2024-01-17, 2024-08-03, 2023-12-14] [] [2024-02-18, 2024-02-18, 9999-12-31, 2024-01-09, 2024-01-08, 2025-02-17, 2009-11-24, 9999-12-31, 9999-12-31, 2024-06-30] [] [] \N [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00] [] [2025-06-18 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00, 2023-12-10 00:00:00, 2003-05-13 00:00:00] [2027-01-16 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-01-17 00:00:00] [] [2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00] [2023-12-19 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00] [2024-07-01 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00] [2026-02-18 00:00:00, 2023-12-14 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00] +132 -2147483648 32679 -825768822 352405591 2024-08-03 2027-01-09 2024-07-01 2023-12-15 no n ? - -- 孟磊公路开头查杀免费不错当地结账提到货 his 1 0 -1.200000 1.200000 14.126100 40.123000 \N 2024-01-31 00:00:00 2023-12-13 00:00:00 2026-01-18 00:00:00 2026-02-18 00:00:00 2025-02-18 00:00:00 9999-12-31 00:00:00 2027-01-09 00:00:00 [1, 0, 0, 1, 0] [0, 0, 1] [] [1, 0, 0, 1, 0, 1, 1, 1] [7, 1, 6, 35, 0] [] [3, -103, -28, 1, 1, 127, 127, -128] [1, -128, -128, 8, -1, 127, 1, 98, 56, -47] [14437, -22901, -27784, 28654, 32767] [32767, -1, 32767] [32767, 2168, 1, 1, 32767, 32767, -1, 32767, 0, -26985] [245, -1, 315] [-377208874, -2147483648, -1818347393, 0, 147483648] \N [] [-1271861493, 1638304619, 64401076, -1066010713, -40138344] \N \N [9223372036854775807, -1, 9223372036854775807, 32679, 1] [245, 9223372036854775807, -1, -4148468, 32679, 147483648, 147483648, 3711983] [-852174279, 1, -2] \N [7, 1, 1] [-2, 0, -1, 0, 0, 1951941186, 5, 1] [18.185100000000, 0.000000000000, 21.107900000000, 0.121200000000, 100.020000000000, 300.343000000000, 100.020000000000, 1.200000000000] \N [1.200000000000, 55.026100000000, 7.179600000000, 20.067500000000, 40.123000000000, -1.200000000000, 0.000000000000, 66.079100000000, 100.020000000000, 300.343000000000] [0.000000000000, 100.020000000000, 1.200000000000, 64.086400000000, 40.123000000000, 1.200000000000, -1.200000000000, 39.002700000000] [55.1953, 39.0729, 16.0435] [96.1526, 100.0200, -1.2000] [1.2000, 300.3430, 23.1172, 40.1230, 17.1067, 72.1694, 300.3430, 300.3430, -1.2000, 100.0200] [0.0000, 0.0000, 100.0200] [-1, 1, 40, 65, 300, 1, 100, 50] [22, 40, 1, 1, 72, 40, 84, 0, 100, 46] [67, 64, 40, 34, -1, 40, 300, 1, 40, 300] [35, 24, -1, 61, -1] ["just think my or can", "say", "now", "", "m", "o", "作风", "-"] ["起床没结束正在性格台式机", "?", "ok that's right back on just with", "x", "三件", "日月丰过的好吗旅游系统嵩县探明小妹能力郑中基", "then", "tell", "", "going got this"] ["here ok really just", "look", "?"] ["营销", "will of", "赫尔", "", "-"] \N [] [] ["你好", "?", "-"] ["u", "y", "z", "r", "w", "r", "y", "z"] ["r", "r", "g", "f", "o", "w", "e", "h"] ["r", "b", "j", "g", "g", "b", "i", "q", "m", "i"] [] \N \N [] [2024-01-31, 9999-12-31, 2023-12-12] \N [2027-01-16 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] [] [2011-12-07 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2023-01-15 08:32:59] [2025-02-17 00:00:00, 2023-12-15 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2009-09-23 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00] [2024-08-03 13:08:30, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00] [2024-02-18 00:00:00, 2017-06-27 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [] [2025-02-17 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00] \N [] [2023-12-14 00:00:00, 2024-01-19 00:00:00, 2023-12-16 00:00:00] +133 -992064470 -104477774 -1 -2147483648 2024-08-03 2018-04-11 2024-08-03 2024-06-30 借款单 \N c my they just a back what go my that in 有限焦小南阳中原北高压 建新 okay ? 0 1 95.034000 0.000000 7.102300 -1.200000 2024-01-19 00:00:00 2025-06-18 00:00:00 2011-07-19 00:00:00 2024-07-01 00:00:00 2027-01-16 00:00:00 2023-12-12 00:00:00 2023-12-13 00:00:00 2023-12-09 00:00:00 [1, 0, 0, 1, 0, 0, 0, 0, 1, 0] [0, 1, 0] [1, 1, 1, 0, 1, 0, 1, 1, 1, 1] [1, 0, 0] [-116, -128, 3] [127, 77, -1, -88, 127] [-1, 1, -128, -128, -111] [-1, 1, -57, -1, -103, 22, 127, 118, 127, 3] [-32768, 24575, 32767, -32768, 10010, -1, -11212, -1, 245, 245] [-7232, -4093, -32768, 18892, -21171, -1, 0, -22219] [-755, -1093, -8410, -32768, 14706, 13278, -32768, 32767] [32767, -9172, 18814] [-289932861, -2147483648, -2147483648] [-1, -1, 0, 1, -1004864163] [2147483647, 1, -64560761, -1062791917, 0] [1, 147483648, 1, 0, 1204988290, 147483648, -23132445, -148078431] [147483648, -1, 3727624, -9223372036854775808, 1] [-6694992157587391001, 1, 1] [245, 245, 5080067500039911797, -1, 1] [245, 6604114938230704203, 32679, 32679, -1, -3131293125952202063, -1061193324, 1, 245, -1648610178792115541] [-1, 1, 5] [-1, -134938807, 1] [1, -2, 1607565160, 1, 1, -245830298, 0, 1, -1, -1] [-1, -2, 1, 1, -2, 1, -2, 0] [0.000000000000, 41.051900000000, 6.118600000000, 100.010900000000, 1.200000000000, 40.123000000000, 58.189400000000, -1.200000000000] [92.141700000000, 1.200000000000, 1.200000000000, -1.200000000000, 40.123000000000, 100.020000000000, 100.020000000000, 0.000000000000] [0.000000000000, -1.200000000000, 93.062000000000, 42.124200000000, 14.065800000000] [] [300.3430, 100.0200, 300.3430, 0.0000, 27.0455, 69.1267, 40.1230, 76.1207] [300.3430, -1.2000, 52.0526, 0.0000, 1.2000, -1.2000, 0.0000, 15.1712, 55.1757, 40.1230] [300.3430, 8.1276, 0.0000, 8.1083, 63.0306, 0.0000, -1.2000, -1.2000, 100.0200, 27.1801] [-1.2000, -1.2000, 21.1020, 1.2000, 98.1520, 40.1230, -1.2000, 1.2000, 40.1230, 91.0438] [40, 80, -1, 52, 300, -1, 100, 15] [100, 7, 40, -1, 0] [1, 0, 100, 40, -1, 69, 100, 59, 100, 84] [0, 1, 1] \N ["英特手机送去安富允许", "视野也许资格", "?", "-", "-", "it i he got", "now", "有波", "mean", "in just about ok as if what"] ["", "could mean he on", "they the some this my not so it", "up", "-", "财务瑞贺咱用是你或据对", "we", "you", "-", "-"] ["最近信天游模拟", "来访", "", "think", "-"] ["企业版收尾百圆早着呢煤炭浪费两台妹妹", "所谓", "say"] ["资料张毅起床本科南天不行时间烟厂七匹狼万兴", "got with ok all they know you're", "就是", "?", "-", "一家", "look", "up", "杂请你们", "?"] [] ["that's", "?", "be back", "don't think if really on in", "?", "-", "", "", "尊重", "院内"] ["g", "u", "g"] [] ["o", "k", "b", "o", "m", "x", "x", "w"] [] [2017-11-01, 2024-01-09, 2023-12-11, 2025-02-18, 2024-01-17, 2013-05-10, 2023-12-20, 2014-05-14] [2023-12-09, 2025-06-18, 2023-12-17, 2023-12-14, 2025-06-18, 2024-08-03, 2014-08-12, 2024-01-08, 2023-12-19, 2026-01-18] [2023-12-13, 2024-01-08, 2023-12-13, 2024-02-18, 2023-12-12] [2024-01-09, 2025-06-18, 2014-08-12, 2027-01-16, 2023-12-11, 2024-01-09, 2024-01-19, 2023-12-09, 2025-02-18, 2024-01-17] [2023-12-17 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00, 2023-12-20 00:00:00, 2024-01-19 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00] [2025-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02, 2024-07-01 00:00:00] [] [9999-12-31 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00] [2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-11 00:00:00] \N [2023-12-15 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-14 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00] [2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [2025-06-18 00:00:00, 2023-12-13 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00] [2013-10-08 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 2024-01-17 00:00:00] +134 0 608522373 -2147483648 -2147483648 2024-08-03 2023-01-15 2026-02-18 2023-12-09 in know that's in in mean are really 计划十五风筝盛唐这种深入数据带中关村 i ? c 看你日月丰当作姚明畅谈志强总计牌子认证 i you're can't but got well 兰州还要电视台关于梁英 1 0 40.123000 1.200000 100.020000 40.123000 \N 2023-12-10 00:00:00 2024-07-01 00:00:00 2023-12-14 00:00:00 2024-02-18 00:00:00 2027-01-09 00:00:00 2023-12-20 00:00:00 2024-01-08 00:00:00 [1, 1, 1] [1, 0, 1, 0, 1] [0, 1, 0, 1, 1, 0, 0, 0] [0, 1, 0, 0, 0, 1, 1, 0, 0, 1] [2, -1, 1, -128, 127, -1, 6, 127, 2, 17] [-1, -128, 2, 127, -47] [-1, 127, 127, -75, -128] [-41, 37, 73, 1, -128] [] [32767, 0, -1, 1, 0, 1, -32768, -18919] [] [0, -16429, 1] [2147483647, 147483648, 1, 0, 702595356, 615282602, 1817298005, 2147483647] \N [-1865975852, 32679, -1, -919091498, 1755154917, 2147483647, 1383150904, 1, 147483648, 1] [-479861257, 32679, 1709330295] [534260743, 0, -1, 0, 2120936415, 6626560866178814370, -9223372036854775808, 32679] [1, 338993, -9223372036854775808, -1907946641467077113, 0, 9223372036854775807, -1443245811292943399, -1, -6480243298573413001, 1628940149] [0, 147483648, 147483648, 6292265133127336513, 0] [32679, 9223372036854775807, -9223372036854775808, 0, 0] \N [-1, -1, -1, -1, 1, 1, 0, -1] [1, 4, -1798918856, -1, -2] [-1, 6, 1, 0, 1, -2, 9, 0] [53.082000000000, 1.200000000000, 56.014200000000, 13.137200000000, 0.000000000000, 28.166000000000, 40.123000000000, 0.000000000000, 1.200000000000, 62.032000000000] [-1.200000000000, 100.020000000000, -1.200000000000, 53.026600000000, 10.091200000000, 40.123000000000, 300.343000000000, 40.123000000000] [78.153000000000, 69.071900000000, 53.009700000000, 0.000000000000, 0.000000000000, -1.200000000000, 300.343000000000, 100.020000000000, 0.000000000000, 300.343000000000] [300.343000000000, 0.000000000000, 83.096200000000, 100.020000000000, 100.020000000000, 300.343000000000, 1.200000000000, 51.168800000000] [63.0431, 1.2000, 300.3430] [40.1230, -1.2000, 40.1230, 33.0138, 40.1230, 40.1230, 3.0826, 40.1230, 40.1230, 300.3430] [87.0587, 64.1438, 1.2000, 75.1838, 100.0200, 300.3430, 0.0000, 1.2000] [40.1230, 86.1758, 0.0000, 1.2000, 40.1230] [300, 38, 0, 300, 88] [0, 100, 100, -1, 1] [56, 0, 1, 100, 40, 40, 100, 55] [0, 100, 78, 40, 74, 40, 79, 100, 9, 300] ["--", "长江路", "the", "hey and", "?", "说不态度花月夜周鹏", "had what him", "time him that's"] ["u", "--", "just", "--", "昨天到达", "交局五级建议透明批发吃饭电脑科技驱动器长期", "严重", "who oh been were you're", "could he well go", "because"] ["--", "-", "新宇", "will about got who he now", ""] ["I'm just hey", "w", "-", "could what all now tell see his", ""] ["小型机环保局后一个鹤壁", "it's", "书生保卡", "当然爱系统集成商第六价格上怎么了同志们", "--"] ["分开怎么了反对回龙太原漏洞", "还有不信出团天数楚留香表情绝对餐费价格", "", "不至于", "a", "广角", "杨宗", "", "k", "下着"] [] [] ["j", "l", "q", "m", "c", "t", "r", "x"] ["w", "f", "n", "p", "o"] ["x", "q", "p"] ["s", "w", "k"] [2024-02-18, 2024-08-03, 2023-12-15, 2024-01-09, 2024-01-17] [2023-12-11, 2023-12-19, 2023-12-13, 2023-12-11, 2024-02-18, 2024-02-18, 2023-12-17, 2027-01-09, 2023-12-10, 2024-01-17] [2023-12-18, 2024-07-01, 2023-01-15, 2024-01-31, 2024-06-30, 2024-01-17, 2024-01-31, 2026-01-18, 2024-02-18, 9999-12-31] [2023-12-10, 2024-01-17, 2024-07-01, 2024-07-01, 2025-02-17, 2024-01-31, 2025-02-17, 2025-02-18, 2025-06-18, 9999-12-31] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00] \N [2024-02-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00] [2025-02-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00] [2023-12-14 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-07-01 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2023-12-12 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 9999-12-31 00:00:00] [] [] [2014-08-12 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00] [2024-07-01 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2024-01-08 00:00:00] [2023-12-11 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00] [] [] +135 353732926 -968473959 32679 147483648 2024-01-31 2024-01-17 2024-07-01 2023-12-20 ? this his what but we your \N - -- 刘雪支原体关系格式总之把控交到 he's will had my 0 0 100.020000 40.123000 9.168800 8.194200 2024-01-19 00:00:00 2024-01-17 00:00:00 2024-02-18 00:00:00 2023-12-16 00:00:00 2027-01-09 00:00:00 2027-01-16 00:00:00 2026-02-18 00:00:00 2023-12-11 00:00:00 [0, 1, 1, 1, 1] [1, 0, 1, 1, 0] [] [1, 1, 0, 1, 0] [-1, 127, 6, 127, 0, 1, 127, -128, -1, -11] [-128, -128, 127, 1, 127, 0, -1, -14] [127, -103, -1, 9, 5, 6, 127, 90, 127, -125] [2, 7, 34] [1, 5127, -32768] [1, 1, 22881, 5593, -27230, -32768, -25072, 0] [11413, 0, 32767, 32767, 28307, 2086, -32768, 0] [1, 3049, 32767, 15894, 12057] [-2147483648, -651880853, -46931707] [1264528237, 2147483647, 352141749] [-1447650772, 2004035382, 703767048] [-1, -1910848469, -881564813, 0, 0, 147483648, -1831304361, 2147483647] [-3461775147751531675, 2744223, -1429351] [-7013535840660513339, 3535647, 147483648, 1, 32679, -1, 6136978, 147483648, 1, -1] [] [-1, 407591453, 7115027996477499013, 147483648, -2774195181202669271] [1, -1, -2, -1, -1, -1, -1, -2] [1, -237328503, -1] [-2, -2, 1446059696] [-1, -2, -2] [21.168100000000, 41.003200000000, 1.200000000000, 76.099800000000, 100.020000000000, 45.173900000000, -1.200000000000, 81.096600000000, 1.200000000000, 40.123000000000] [55.029700000000, 1.200000000000, 0.000000000000] [8.013700000000, 1.200000000000, 40.123000000000, 1.200000000000, 100.020000000000, -1.200000000000, 12.048000000000, 300.343000000000, 40.123000000000, 35.153300000000] [1.200000000000, 1.200000000000, 40.123000000000, -1.200000000000, 23.153200000000, 68.072600000000, 40.123000000000, -1.200000000000, 0.000000000000, 1.200000000000] [40.1230, 1.2000, 40.1230, 68.0195, 0.0000, 1.2000, 0.0000, 100.0200] [-1.2000, 1.2000, 13.0005, 300.3430, 1.2000, 300.3430, -1.2000, -1.2000] [52.0850, 40.1230, 35.0832, 1.2000, 98.0095] [0.0000, 100.0200, 82.1259, 27.1527, 100.0200] [0, 300, 1, 0, 0, 0, 40, 4, 0, 1] [-1, 60, 300, 5, 1, 43, 5, 34] [39, 100, 100, 96, 96, 67, 100, 300] [40, 1, 300, -1, 1, -1, 40, 16] ["yeah not is been from then go good yes", "can", "?", "云隐磁盘柜邮寄事务所明威安排需方核对", "五笔", "--", "最起码孙娟关闭郑州市东大街怎么着热线", "-", "排名智者", "-"] ["do", "not really", "--", "兼容讨论组金信", "这段"] ["on how him your were see had all were", "?", "--", "c", "下雨", "-", "-", "q", "is", ""] ["--", "--", "", "was is from he look well would look", "老公下班说不清楚国虹带你"] ["", "at here I'm it there who are have did", "带来", "现代少秋", "not could who time did right", "", "really", "?"] ["--", "医院铿锵双机经销商列表指出来", "him well will he's I'm like he it something"] [] [] ["z", "y", "o", "k", "p", "e", "b", "b", "z", "w"] [] [] ["j", "d", "i", "h", "c", "k", "d", "x", "d", "l"] [2023-01-15, 2023-12-12, 2025-02-17] [2023-12-20, 2024-08-03, 2025-06-18, 2023-12-12, 2027-01-09, 2025-06-18, 2027-01-16, 2026-01-18] [] [2024-08-03, 2023-01-15, 9999-12-31, 2024-02-18, 2024-02-18, 2023-12-17, 2023-12-14, 2023-12-20] [2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59, 2004-05-06 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59] [2024-01-08 00:00:00, 2023-12-15 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00] [2024-02-18 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00] [2024-07-01 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2024-01-19 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00] [2025-02-17 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00, 2023-12-12 00:00:00] [9999-12-31 23:59:59, 2014-08-12 00:00:00, 2023-12-11 00:00:00] [2023-12-11 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2024-07-01 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00] [2023-12-10 00:00:00, 2014-08-02 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00] [2027-01-16 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00, 2023-12-19 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00, 2023-12-09 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00] [2024-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00] +136 1504874742 2147483647 -1 829093298 2023-12-19 2023-12-14 2026-02-18 2027-01-16 -- one or some why say the do - you're 没有到货小计游戏关于进去每天 can you it not with then on would he's a 郭晓报名下午最大兰州青青西欧啊一块群主舞阳 think 1 1 59.002200 35.007000 22.062000 -1.200000 2024-07-01 00:00:00 2024-02-18 00:00:00 2024-01-17 00:00:00 2025-02-17 00:00:00 2025-06-18 00:00:00 2025-02-18 00:00:00 9999-12-31 23:59:59 2024-01-08 00:00:00 [0, 1, 0, 0, 0, 1, 1, 0] [0, 1, 0, 1, 1] [1, 0, 1] [1, 0, 1, 1, 0, 0, 1, 1] [57, 127, -128, 1, -85, -128, 8, 1, -47, 0] [-31, 127, -128, 127, 127] [1, 106, 104, 0, 24] [127, 1, -1, -1, -61, -118, -117, 1, -14, -1] [] [32767, 29395, -1929, 32767, 2085] [-1378, 0, 24729, 32767, 17209, 245, -9288, 245] [-18445, -23118, -328, 245, 24232, -32768, 15479, 22719] [-1, -2147483648, -172176387, -2147483648, 1692610939, -615450109, -1376859202, -326624671] [-1376651860, -2147483648, -610117723, 318245135, 2147483647, -1, 0, 1] [-296559717, 319778717, 2022831264, 1339614070, -1369040817, 0, 1600318367, -2147483648, 0, 0] [0, 1, -2147483648] [-9223372036854775808, 8899850781244562602, -9223372036854775808, 245, -483652500] [1, 245, -1309566] [] [3058908850270247437, 147483648, 32679, -8468432761114611712, -9223372036854775808, 1447193283, 1518120792, 147483648, 547294083, -2043749780] \N [1, 7, -2, 1, 2] [] [-1, -2, 743947260, 0, -1, -2, 0, -1, 0, 0] [1.079900000000, 1.200000000000, 0.000000000000, -1.200000000000, 100.020000000000, 1.200000000000, 1.200000000000, 100.020000000000] [300.343000000000, 300.343000000000, 40.123000000000] [100.020000000000, 85.187500000000, 100.159800000000, 300.343000000000, 300.343000000000, 1.200000000000, 14.119500000000, 0.000000000000, 67.079200000000, 300.343000000000] [14.093800000000, 0.000000000000, 100.020000000000] [40.0259, 32.1431, 74.1151, 56.0277, 1.2000, 13.1169, 11.1478, 100.1557, 1.2000, 300.3430] [50.1570, -1.2000, 70.1522, -1.2000, 70.0408, 41.1541, 1.2000, 300.3430] [] [40.1230, 82.0635, 15.0655] [] [-1, 100, 7, 40, 10, 0, 1, 23, 40, 100] [300, -1, 1, 100, -1] [] ["", "my", "关系人家"] ["?", "英文", "how", "think", "are come look he's"] ["?", "有机会不对战友责任翔龙名单天成苑这么做", "朦胧", "?", "--"] ["", "get", "动作透明元整出入", "三门峡肥肉你们信源下次吧外教方面", "l", "?", "your", "can you're really I'll look that on mean", "--", "-"] ["蠢材", "just", "just or but me well or i", "下一部分", "-"] ["五级", "-", "and", "-", "移动", "大姐", "?", "been so don't the"] [] ["", "you're to then tell", "晓得光纤说声这辈子授易设备这几太难"] ["f", "j", "m", "t", "c", "l", "a", "p", "t", "q"] ["y", "y", "l"] ["u", "y", "v", "l", "u", "o", "x", "r"] ["v", "r", "u", "f", "c"] [2024-06-30, 2024-02-18, 2027-01-16, 2024-08-03, 2023-12-09, 2023-12-18, 2023-12-09, 9999-12-31, 2023-12-15, 2023-12-15] [2026-02-18, 2026-01-18, 2025-06-18] [9999-12-31, 2024-01-19, 2025-06-18, 2024-01-17, 2024-01-31, 2023-01-15, 2023-12-12, 2023-12-09, 2027-01-16, 2023-12-09] [2025-06-18, 2023-12-14, 2023-12-16, 2026-01-18, 2025-06-18, 2023-12-18, 2024-02-18, 2023-12-12, 2024-07-01, 2025-06-18] [2024-01-31 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02, 2024-08-03 13:08:30] [2024-02-18 00:00:00, 2024-06-30 12:01:02, 2027-01-09 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00] [2024-01-08 00:00:00, 2025-02-18 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00] [] \N [2023-01-15 08:32:59.123000, 2023-12-10 00:00:00, 2025-06-18 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123000, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2014-08-12 00:00:00] [9999-12-31 23:59:59, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2014-05-10 00:00:00, 2014-08-12 00:00:00] [2024-01-19 00:00:00, 2023-12-16 00:00:00, 2024-01-19 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00] [2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00] [] [2023-12-12 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00] +137 \N 359511553 32679 0 2026-01-18 2024-02-18 2024-02-18 2024-02-18 还有你的 七匹狼 because tell what oh all i it from he good didn't on you get as be ? time hey all something what 物流 腾龙天威 1 0 17.170700 40.123000 75.160100 1.091400 2023-12-19 00:00:00 2023-12-14 00:00:00 2023-12-09 00:00:00 2023-01-15 08:32:59.123000 2024-01-17 00:00:00 2025-02-18 00:00:00 2023-01-15 08:32:59.123123 2024-01-19 00:00:00 [0, 1, 0, 1, 1, 0, 1, 0, 1, 1] [1, 1, 1] [0, 0, 1, 1, 1, 0, 0, 1, 1, 0] [1, 0, 1, 1, 1, 1, 1, 1, 0, 0] [0, 119, -17] [] [] [-93, 3, -128, 0, -128, 117, 77, -49] [-13108, 32767, 0, 1, 245, 245, -3294, 32767, 245, 1] [1, -5172, -1, -26905, 12133] [32767, -32768, 1, -1150, 10106, -32768, 7747, -1, -8097, 22701] [-32768, 17872, 5762, -32768, 10235, -1, -24317, -32768, 28782, -21181] [-2147483648, -2147483648, 770219233, -104709724, 699648373] [32679, 0, 1, 1, -951219723, -1, 32679, 0] [] [] [-1, 9223372036854775807, 147483648, -6764068354604536077, 32679, 9223372036854775807, 1, 0, 32679, 245] [147483648, -9223372036854775808, -8609750594479054778, -5871155, 245, 32679, 3462396, 1094405049012982896, 245, -1385597038] [1, 1, 7436614716457955410, -8381743267656080286, 2924881960056731953] [9223372036854775807, 3092504023923320789, -1, 245, 1433472456] [-1, -1, 1, 0, 4, 1, 0, 1] [1, 1, 1, 1, 1] [] [] [300.343000000000, -1.200000000000, 94.127800000000] [89.089600000000, 45.063000000000, 0.000000000000] [] [40.123000000000, 31.038300000000, 0.000000000000, 34.185200000000, 100.020000000000, 1.200000000000, 100.020000000000, -1.200000000000, 300.343000000000, 0.000000000000] [] [40.1230, 41.1118, -1.2000, 300.3430, 40.1230, 0.0000, -1.2000, 0.0000] [1.2000, 12.1523, 1.2000, 40.1230, 40.1230] [-1.2000, 40.1230, 40.1230, 49.1675, 0.0000, 40.1230, 56.0729, 300.3430] [40, 26, 33, -1, 40, 86, 40, 100] [40, 40, 1, 100, 82] [1, 69, 31, 0, 0, -1, 40, 1, 66, 51] [92, 40, 63, 36, 90, 0, 50, -1, 66, 1] ["?", "--", "鹏飞朦胧类似做不了交叉口哦顺河路", "锐捷", "?", "--", "this", "--", "-", "ok not to mean my"] ["?", "不能说", "to", "for", "操作询问不行呀帮忙卫华生活中群管理员不高跃民", "--", "right", "his", "便宜特此证明新的无锡开标刘金谈下来", "自己河南不在小故事一套杀毒不少等着客运中科"] ["", "l", "-", "", "I'm okay got see see"] ["", "when ok not they well", "-", "x", "get", "中华", "做人信息部兼容大姑感触你好呀", "陈文杰大家六万日信"] ["?", "?", "多家供奉路值得岂能经济参加三阳第二次"] ["支原体高性能失误", "z", "know and yeah did"] ["算数", "-", "at", "from", "u"] ["all", "衣服同力不在一线要不授权经销总代理", "we", "ok", "-"] ["k", "u", "y"] ["b", "a", "s", "q", "c"] ["u", "t", "m", "k", "s", "q", "k", "y"] ["c", "x", "e", "q", "r", "g", "d", "e"] [] [2023-12-18, 2023-12-09, 2024-06-30, 2027-01-09, 2023-01-15, 2023-12-10, 2025-06-18, 2023-12-12] [2023-12-11, 2026-02-18, 2023-12-15, 2026-02-18, 2024-02-18, 2024-01-09, 9999-12-31, 2023-12-19] [2025-02-17, 2025-06-18, 2026-02-18] [2023-12-16 00:00:00, 2013-07-28 00:00:00, 2024-01-19 00:00:00, 2001-06-11 00:00:00, 2027-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00] [] [2011-05-10 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 00:00:00] [] [] [2023-12-16 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30, 2023-12-18 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00] [2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00] [] \N [2023-12-20 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00] [] [] +140 32679 2147483647 32679 1904490382 2023-12-18 2027-01-09 9999-12-31 2024-01-19 - y 检测埋在 here that ? 参数 折扣 0 1 -1.200000 \N 99.154500 40.123000 2023-12-17 00:00:00 2023-12-20 00:00:00 2025-02-17 00:00:00 2023-01-15 08:32:59.123000 2023-01-15 08:32:59.123123 2025-02-17 00:00:00 2024-01-31 00:00:00 9999-12-31 00:00:00 [0, 0, 1, 0, 0] \N [] [0, 0, 0, 0, 0, 0, 1, 0] \N [127, -1, -66, 0, 89, -9, -128, 1] [-123, -117, 127] [-7, 127, -104, -1, -1, -128, 0, 6] [32767, 0, -27423, 32767, 245, 32767, -31290, -16005] [245, -13533, -1] [-3443, -9328, -13277] [] [1539092017, -482493537, 2147483647, 32679, -269149078, -897703151, 2147483647, -1924829351] [147483648, -1370656090, -1622814775] [2022840163, -981990980, -797308383, -1, 2147483647, 1137672815, -440700668, 94039843, -1054749121, 147483648] [-53653535, -1, 516129011, 147483648, -367360517] [1, 245, 8655456191135556679, 245, 245] [2008131577, 5348750614930894181, -3591315286080563315] [] [0, 1174478934, 147483648] [-1847280092, 1488640623, 1, 0, -525892423] [0, 4, -2, -1, -2] [0, -2, -1, 4, 1, -1, 0, -1, 7, 6] [-1770351412, -2135336823, -1] [1.200000000000, 100.020000000000, 18.087000000000, 65.166300000000, 92.180700000000, 300.343000000000, 1.200000000000, 100.020000000000] [300.343000000000, 300.343000000000, 54.092000000000, 82.148900000000, 1.200000000000] [91.129400000000, 40.123000000000, -1.200000000000, -1.200000000000, 100.170500000000, -1.200000000000, 16.106800000000, -1.200000000000] [0.000000000000, 40.123000000000, 300.343000000000, 59.195800000000, 2.106900000000] [15.1546, 300.3430, 100.0200] [51.0525, 100.0200, 1.2000, 1.2000, 0.0000, 52.1125, 0.0000, 0.0000, 40.1230, 69.0564] [1.2000, -1.2000, 44.0605, 75.1366, 40.1230, 23.0673, 43.0100, 300.3430, 0.0000, -1.2000] [] [67, 300, 77, 0, 100, 75, 85, 32] [0, 17, 72, 15, 100, 300, 300, 300] [54, 83, 40, 300, 0, 20, 73, 40, 86, 76] [1, -1, 300, -1, 49, 1, 23, 38] [] ["", "嘉运", "--", "", "t"] ["ok", "旅游下着机关动作这种刘总士大夫", "", "not", "", "美丽", "?", "p"] ["", "that", "can really to so", "m", "火车对话框结账", "", "at", "向你", "--", ""] ["需求", "her now", "--", "多条", "?"] ["--", "that's think been", "你吃法科技大厦何时", "-", "充满光缆授权回忆集团凯峰作风一辈子财务部"] [] ["is your going yes", "but look we yes okay had up how can't", "-", "u", "-", "着呢区分正确半天班可笑你不系统集成有些", "责任", "what", "y", "?"] ["n", "d", "x", "d", "x", "j", "x", "u"] ["x", "x", "x"] ["z", "t", "i"] ["h", "f", "g", "h", "r", "b", "m", "y", "u", "z"] [2023-12-15, 2024-01-09, 2025-06-18, 2023-12-15, 2023-12-19] [2023-12-15, 2026-02-18, 2024-01-17, 2025-06-18, 2023-12-20] [2025-06-18, 2027-01-16, 2027-01-16, 2027-01-16, 2023-01-15, 2024-01-19, 2026-02-18, 2024-01-17] [2024-02-18, 2024-01-31, 2023-12-20, 2023-12-18, 2023-12-14, 2025-06-18, 2023-12-18, 2023-12-11, 2023-01-15, 2024-01-19] [2024-08-03 13:08:30, 2024-01-19 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59] [] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00] [] [2023-12-19 00:00:00, 2024-01-19 00:00:00, 2014-08-12 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2024-01-08 00:00:00] [2024-02-18 00:00:00, 2018-05-05 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00] [2023-12-13 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00] [] [2023-12-13 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [] [] +142 147483648 -727182314 528374581 147483648 2023-12-14 2024-06-30 2024-06-30 2023-12-12 -- - 刑阳 or get think with one or come it and this 天华 - they be at oh she \N 1 100.020000 -1.200000 0.000000 40.123000 2023-12-13 00:00:00 2023-12-17 00:00:00 2023-12-15 00:00:00 2024-02-18 00:00:00 2023-12-11 00:00:00 2024-07-01 00:00:00 2023-12-09 00:00:00 2023-12-12 00:00:00 [1, 1, 0, 1, 0, 1, 0, 0] [1, 1, 1, 0, 0] [1, 1, 0] [0, 0, 0, 0, 0, 0, 0, 1] [43, 127, -128, 0, 1] [] [-1, -128, 1, -128, 32, -128, -1, 54, -35, -111] [] [32767, 19736, -32768, 4948, -15361] [-32171, -1, -9496, 30265, 17837, 30712, 29, -1, 0, 245] [] [-13756, 11207, -16381, -18782, 1] [697535200, 1, 0, 1026572109, -964875522, -1, 1954541233, 32679] [147483648, -2128043689, 147483648, -920888249, 1, 1077671586, 147483648, 2147483647, 1976159579, -106900470] [-2147483648, 2147483647, 147483648, 655234343, 0, 32679, 1453674146, 2100386695] [147483648, -1, 147483648, -924505646, -2147483648, 442198781, 2147483647, 0] [147483648, -1, 147483648, 0, 1607970429119847646, 245, -1, 667818241, 1, 147483648] \N [] [-1090790418658982892, 9223372036854775807, 9223372036854775807, 9223372036854775807, 7661817956669579061, 245, 0, 949142590] [47429162, 7, -2, 744314167, 6] [1, 1, -2, -2, 8, 0, -1260389960, 0] [] [1, -1, -2, 1930011586, -2, 1, 4, 117934464] [81.051000000000, -1.200000000000, 0.000000000000] [0.000000000000, 300.343000000000, 40.123000000000] [83.004200000000, 31.112100000000, 13.061100000000, 37.100500000000, 300.343000000000] [-1.200000000000, 92.156600000000, 6.177300000000] [1.2000, 300.3430, 0.0000, 0.0000, 1.1717] [] [1.2000, 22.0685, 100.0200, 300.3430, 71.1661, 23.1384, 36.1917, 16.0687] [40.1230, 36.1279, 35.1821, 92.1835, 300.3430, 40.1230, 0.0000, 74.1537] [100, 88, 64, 64, 100, 14, 27, 52] [100, 51, 0, 2, -1] [] [300, 96, 1, 0, 52] ["?", "--", "z", "o", "服务器站", "--", "her back for how some the mean have good hey", "--"] ["调货", "--", "there go up they well that's", "检查手机号一些", "look"] ["--", "--", "", "in it it's from now as what", "be", "燕龙", "then as oh will he", "on if get don't with can just here"] ["because do mean what oh could it's back was", "on when as not this at yeah it when yeah", "how would back tell don't of were see", "have like if", "?", "it", "-", "would"] ["", "--", "here", "y", "q", "when", "-", "天下"] ["项目行业宣传金信未来用友投影机不便冤枉湖北", "hey", "t", "me", "拜访晓燕新版"] ["--", "in", "-", "t", "又给", "but on time ok want a get didn't", "误区深爱比例技术好心", "in is not", "go", "had"] ["有过", "-", "燕龙发吧客户群金牌总代理货到美好能力跑跑卡丁车", "分析正道聚兴你怎么去的协议春节做过拿货", "-", "of", "just hey I'll oh going at with good don't not", "--", "he's", "-"] ["d", "s", "d", "p", "b", "y", "k", "g"] ["e", "l", "y", "o", "l"] ["t", "g", "x", "f", "a", "g", "m", "m", "n", "v"] ["r", "h", "u", "u", "r"] [2023-12-16, 2027-01-16, 2023-12-16, 2024-01-31, 2023-12-14] [2023-12-15, 2023-12-20, 2023-12-12, 2023-12-16, 2024-02-18, 2023-12-18, 2023-12-09, 2024-01-08] [2024-02-18, 2024-01-19, 2023-12-20] [2023-12-13, 2024-02-18, 2024-02-18, 2023-12-18, 2025-02-17, 2023-12-16, 2024-02-18, 2023-12-11, 2023-12-18, 2023-12-13] [2025-06-18 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00] [2024-08-03 13:08:30, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-20 00:00:00] [2024-01-19 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2023-12-19 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00] [2023-12-16 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00] [2023-12-19 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 2024-07-01 00:00:00] [2027-01-09 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2023-12-18 00:00:00, 2004-08-05 00:00:00] [2023-12-12 00:00:00, 2001-04-20 00:00:00, 2027-01-16 00:00:00] [2023-01-15 08:32:59.123000, 2023-12-14 00:00:00, 2024-01-17 00:00:00] [2023-12-10 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-09 00:00:00] [2024-02-18 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2023-12-18 00:00:00] [2025-02-17 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00] [2024-08-03 13:08:30, 2023-12-18 00:00:00, 2023-12-13 00:00:00] +146 -755508609 1921820868 -2147483648 0 2023-12-13 2025-02-17 2024-02-18 2024-08-03 梦幻水族若虚风格好人飞扬南阳倪玲梅词句提醒天宇 so time oh 全国 - me ? 0 0 -1.200000 40.123000 1.200000 0.000000 2025-06-18 00:00:00 2023-12-10 00:00:00 2026-02-18 00:00:00 2023-12-10 00:00:00 2014-08-12 00:00:00 2025-02-18 00:00:00 2023-12-14 00:00:00 9999-12-31 23:59:59 [0, 1, 0, 0, 0, 1, 1, 0] [0, 1, 1] [1, 1, 1, 0, 1, 1, 0, 1] [0, 1, 1, 0, 0, 1, 1, 0] [0, -1, -51, 0, 0] [7, -128, -76] [-128, 53, -87, -92, -1, 2, 1, -128] [0, 95, -91, 124, -1, -1, 58, 127, 127, 1] [21611, -21279, 32767] [] [1, -17199, -32768] [-32768, 25418, -9176] [147483648, 726684868, 147483648] [-113312300, -611601031, 0] [-479640255, -710656539, -2147483648, -283625183, 2147483647] [-1, -1330598639, 0, 1491630244, 147483648, 0, 1288612976, 0] [-1, 1542019, 245, 1552591601, 1, 32679, 147483648, 32679, 3419274681111459554, 0] [-5860278060479765157, -797305577, -1, 5152322833406541516, 6577356662398493759, 9223372036854775807, 1, 8372637776986943381] [-2836444906321554696, 4370720144607610618, 32679, -756612080083206931, -3853595] [-1, -1, 9209148637650824119, 136214228, -2152930, 245, 8927410306135636468, -2985477, 0, 9223372036854775807] [-2, 1, 0] [-1, 1583577985, 1] [1523197414, 0, -1] [0, 1, -1] \N \N [] [40.123000000000, 35.007800000000, 29.122500000000, 300.343000000000, 16.073800000000, 18.138600000000, 40.123000000000, -1.200000000000] [1.2000, 1.2000, 100.0200, 100.0200, 18.0392, 1.2000, 87.1817, 100.0200] [49.0433, 50.1946, 40.1230] [49.1632, 63.1738, 29.0079, 300.3430, -1.2000] [58.1341, 32.0670, 100.0200, 0.0000, 96.0827] [40, 300, 300, 89, 90, 73, 100, 0, 300, 100] [40, 300, 40, 81, 40, -1, -1, 1] [58, 100, 18, 52, 0] [] ["在家里", "look so now as on say there don't yeah one", "购买专区"] ["back", "get had had on some yeah here", "--", "or me about come some", "me or can't is here there can't look say back"] ["", "for ok i time as back is", "-", "of like or", ""] ["w", "实实在在年后纯平", ""] ["-", "", "?", "?", "l"] [] ["--", "这也是", "be", "做完恶心死机架算数随北风光通路诊断不容易在乎", "-", "?", "治疗那这次心里组哟广告周期确定蠢材", "yes going I'll a can't could", "新款", "--"] ["山下", "q", "some my like here time well as", "-", "夜间回访网管中心多大委托收款五万条艺术"] ["p", "q", "z"] ["x", "d", "u"] ["j", "c", "s", "f", "g", "v", "h", "s"] [] [2024-02-18, 2023-12-15, 2023-12-16] [2024-01-19, 2024-01-08, 2026-01-18, 2023-12-09, 2024-08-03, 2024-02-18, 2026-02-18, 2026-01-18, 2023-12-12, 2023-12-20] [2025-06-18, 2023-12-20, 2024-02-18, 2025-06-18, 2024-01-09] [2024-01-31, 2023-01-15, 2014-08-12, 2023-12-12, 2025-02-18, 2024-01-08, 2023-12-14, 2024-02-18, 2014-08-12, 2023-12-16] [2024-02-18 00:00:00, 2024-06-30 12:01:02, 2023-12-19 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] [] [2024-02-18 00:00:00, 2023-12-19 00:00:00, 2025-02-18 00:00:00] [2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00] [2027-01-16 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00] [2023-12-15 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59] [9999-12-31 23:59:59, 2025-06-18 00:00:00, 2023-12-12 00:00:00] [2024-08-03 13:08:30, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2017-04-19 00:00:00, 2024-08-03 13:08:30] [2023-12-12 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-13 00:00:00] [] +147 1177427669 147483648 147483648 -290769086 2023-12-14 2024-06-30 2025-06-18 2023-12-13 串货 老乡低端明辉大厦桌子寻找定金慧眼人员名单诊断 有个 \N 信息化 some then i going his oh are say it i with 0 0 0.000000 300.343000 100.152600 6.125900 2023-12-20 00:00:00 2023-12-16 00:00:00 9999-12-31 00:00:00 2024-02-18 00:00:00 2025-06-18 00:00:00 2027-01-16 00:00:00 2025-02-18 00:00:00 2024-02-18 00:00:00 [0, 1, 1, 0, 0] [1, 1, 1] [1, 1, 1, 0, 0, 1, 0, 1, 1, 0] [] [4, 7, 127, 0, 62, 1, 0, 1, 0, -128] [-1, -128, -8, -125, 7, -47, 1, -128, -128, 127] [-1, 1, 127, 1, -1] [-128, 1, 104, 127, -128] [245, -1, -1] [-9520, 20599, 16912, 11224, 14053, 245, -8397, 32767] [-32768, -16880, 3855, 0, 27975] [28840, -32768, -18065, -1, -15589, 1973, 0, 14465, 32767, 31037] [] [-2147483648, 2147483647, 1, -2147483648, 1125182158] [] [] [51353572698702141, 32679, 9223372036854775807, 245, 1018941770, -9223372036854775808, -8473500895312929266, 9223372036854775807, 147483648, -1] \N [] [-3547573107420750131, -3872912203026223981, -4838525, 6742789856775890924, 245, -9223372036854775808, 147483648, -1, 32679, -4383480] [-1, 1, -925969892, -2, -351506173, 1, -1, 1] [1180796425, 0, -2] [-2, -2, 0, 0, 0] [1157903913, -2, 564046107] [23.171200000000, 0.000000000000, -1.200000000000, 92.161600000000, 1.200000000000] [99.035700000000, 0.000000000000, 15.035500000000, 79.068900000000, 40.123000000000] [40.198800000000, 100.020000000000, 1.200000000000, 26.185200000000, 300.343000000000, 81.007800000000, 73.009700000000, 100.020000000000] [] \N [] [1.1539, 61.1774, 64.1727, 0.0000, -1.2000, -1.2000, 74.1955, -1.2000, 100.0200, 14.1438] [] [0, 1, 1] [-1, 14, 300, 100, 300, 300, 91, 100, 40, 300] [] [40, 0, 300, 98, -1] ["签名速度新增歘俩新国", "did i no did think were that's", "him", "can", "y"] ["really", "?", "the think as did I'm out back why", "", "not is this we I'm if him"] ["-", "f", "-", "真正含有照面保险英特顾星", "-", "--", "则很难", "--", "", "伏牛路"] ["can", "a", "星月嘉运达脱机不需要上有入职不咱优势回你哟偶那个", "拿不到", "回你这段数量伟博出现创世纪什么时间刘亚期限"] ["-", "埋在", "y"] ["up", "", "okay not for a", "电话本相见", "-"] [] ["n", "粘贴肥肉场景检验光电", "why this just", "你是", "-", "--", "okay", "know", "哪呢个二姑广告词新年好金信保密还好吧", "she be what want i could something your"] ["c", "r", "h", "i", "j", "q", "k", "y"] ["t", "p", "l"] ["h", "u", "y", "y", "l"] ["u", "s", "m", "s", "v"] [2023-12-13, 2024-01-19, 2023-12-11, 2023-12-11, 2024-01-19, 2024-01-31, 9999-12-31, 2024-02-18] [2023-12-09, 2026-01-18, 2023-12-11, 2023-12-11, 2024-06-30] [2024-02-18, 2024-02-18, 2025-06-18, 2024-01-19, 2026-01-18, 2023-12-11, 2006-12-03, 2023-12-12, 2025-06-18, 2023-12-12] [2024-01-31, 2023-12-12, 2024-02-18, 2024-01-31, 2023-01-15, 2024-08-03, 2023-12-19, 2025-02-18] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00] [2024-01-08 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02] [] [2023-12-13 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2023-12-16 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2023-01-15 08:32:59, 2023-12-09 00:00:00] [] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00] [] [2023-12-18 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-09 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00] [2023-12-16 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00] [9999-12-31 23:59:59, 2023-12-20 00:00:00, 2023-12-12 00:00:00, 2018-05-03 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00] +148 1 \N 32679 2147483647 2023-12-16 2023-12-12 2026-01-18 2024-06-30 考前 - \N \N from have we my ? 业务 not and like just we in yes \N 0 100.020000 40.060400 81.018200 43.181800 2024-01-17 00:00:00 2024-01-31 00:00:00 2023-12-10 00:00:00 2006-04-18 00:00:00 2024-01-08 00:00:00 2014-08-12 00:00:00 2023-12-14 00:00:00 2024-06-30 12:01:02.123000 [] [1, 0, 0, 1, 1] [1, 1, 1, 1, 1] [] [127, 1, -69, 121, 43, 1, -1, -128] [-128, 0, 47, 11, 127, 1, -13, -56] [] [4, 0, 3, 1, -1, 4, -23, -93] [-18998, -21656, 21858, 28957, -29333, 0, 1542, -4504, 245, 26669] \N [32767, -32768, 15188, 30471, -27983, -32768, 23460, 32767] [-1, -13209, -1, 15967, 12569, -32768, -32768, 245] [] \N [32679, -1, -1514563329, -576597458, 829997833, -1, 2008596564, 1813191314, 1649582110, 0] [] [1, -1, 9223372036854775807, -4755117452297618450, 1] \N [-1046129684410059920, -238195891, 9223372036854775807, 1, 3458728249412945533, 0, 245, 1, -7968538393280256916, 9223372036854775807] [-1, -6617948, -7542129272059152758, -2436310240404459608, -9223372036854775808, 0, -1, 1] [] \N [-57692413, -1, 1080062976, 5, 1, 7, 1730739993, 0] [] \N \N [100.020000000000, 300.343000000000, 40.123000000000, 1.200000000000, 1.200000000000, 0.000000000000, -1.200000000000, 0.000000000000, 100.020000000000, 0.000000000000] [] [300.3430, 100.0200, 50.0647, -1.2000, 24.0563, 100.0200, 300.3430, 100.0200] [300.3430, 74.0010, 300.3430, 5.1031, 62.1535, 1.2000, 86.0188, 23.0228, 100.0200, 76.1274] [100.0200, 300.3430, 95.1401, 21.0888, 0.0000, 40.1230, 300.3430, 300.3430] [15.0234, 300.3430, 0.0000] [80, 23, 100, 86, 5, 300, -1, 300, 99, -1] [100, 89, 1, 40, -1] [28, 0, -1, 100, -1, 300, 40, 1] [-1, 15, 0] ["全勤奖子卡", "", "-", "整理双千兆", "see he's up at can so so can", "?", "一小时", "say", "think", ""] ["know", "喜爱企业级", "--"] ["did here to to was hey a", "--", "", "长城那种玫瑰好久科技市场才能", ""] ["?", "-", "him here we i all but it", "v", "", "可选没人圈内心理学万兴创恒银联风筝等会核心总代理", "回报", "数字显示", "偶然", "女孩架子半天班系统乔东看来有个你来吧写吧"] ["i", "有时间派人小灵通经济吃住咩女实力看吧短信着急", "k"] \N ["-", "as what look right on at say I'm okay what", "would", "on yes were", "?", "咱们制度打给必须感觉", "推推", "外地"] [] ["u", "g", "d", "z", "r", "i", "c", "h", "n", "v"] \N ["y", "e", "f", "w", "i"] ["j", "y", "j"] [2026-02-18, 2024-02-18, 2024-02-18] [9999-12-31, 2025-06-18, 2023-12-11, 2023-01-15, 2023-12-12, 2023-12-16, 2025-06-18, 9999-12-31] [2024-01-19, 2023-12-09, 2023-12-14] [2023-12-17, 2014-08-12, 2023-12-14, 2026-01-18, 2024-08-03, 2023-12-10, 2027-01-09, 9999-12-31] [9999-12-31 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2025-02-17 00:00:00] [2023-12-17 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2023-12-11 00:00:00] [2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00] [2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-08 00:00:00, 2023-12-13 00:00:00, 2007-02-12 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2025-02-17 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00, 2024-01-19 00:00:00, 2024-01-08 00:00:00] [2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30] [2024-01-17 00:00:00, 2027-01-16 00:00:00, 2018-04-28 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-15 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2023-12-09 00:00:00] [2024-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-16 00:00:00, 2013-10-09 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-09 00:00:00, 2025-02-18 00:00:00] [2023-12-16 00:00:00, 2024-02-18 00:00:00, 2019-11-28 00:00:00, 2023-12-16 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59] +149 792039574 2076244191 -1867163746 1 2024-06-30 2025-06-18 2024-02-18 2023-12-14 利万图形工作站冲动问世 see would just can't on him him because yes as who going think there really know why the well up of yeah 创彩 你能看到 \N 0 1.200000 0.000000 100.020000 300.343000 2023-12-10 00:00:00 2023-12-18 00:00:00 2023-12-12 00:00:00 2023-12-11 00:00:00 2023-12-15 00:00:00 2023-12-18 00:00:00 2024-01-19 00:00:00 2023-12-12 00:00:00 \N [0, 1, 0, 1, 0, 1, 0, 0] [] [1, 1, 0] \N \N [-60, 0, 0] [1, 0, -63] [-29354, -13679, 26850, 0, -29948] [5741, -32768, -8501, 1, 32767, 27997, -2380, 245] [-23046, 32767, 0, -1, -32253] [20777, -32768, 6440, -30048, -27000, 9558, -1, -3732, 32767, -32768] [] [0, 1274557773, 2130352557, -998268563, -75344019, -1293449042, -156627704, -2147483648] [39122664, 147483648, 1318600326] [] [] [1, 32679, 245, -5855911, -1, 9223372036854775807, 147483648, 1, -1651469857, -6025746666358851406] [7453499, -2271971612626440821, 4464575781673181494, -2893526416372254117, 4393047140260893086, 32679, 2829239299443780111, 6237829710090436139, -6704674, -9223372036854775808] [-2198667778470253767, 32679, -8935256658618146926, -1, -502764104, 1731860712, 1, -3835886264052657205] [-2, 0, 1, -1844767584, 1, 3, -1, 0, 8, 0] \N [5, 0, -1] [0, 5, -1] \N [40.123000000000, 0.000000000000, 1.200000000000] [-1.200000000000, 69.035200000000, 29.048100000000] [40.123000000000, 0.000000000000, 40.123000000000, 0.000000000000, 44.093400000000, 1.200000000000, -1.200000000000, 0.000000000000, 100.020000000000, 1.200000000000] [40.1230, 39.0124, -1.2000, 100.0200, -1.2000, -1.2000, 1.2000, 0.0000] [-1.2000, 30.0235, 100.0200] [1.2000, 300.3430, 1.2000, 100.0200, 1.1383] [300.3430, 0.0000, 40.1230] [64, -1, 31] [] [96, 27, 0] [-1, 0, 40, 1, 23, -1, 100, 1] ["性价比名称直接锐成作风说吧丹尼斯十五", "玉清", "拓展留个山下退信老师动画文艺力天要不然显卡"] \N ["放假", "at", "?", "位置", "now there of the it's some"] [] ["春子也有过航海路", "发到", "--", "are not when right ok no as time see with", "原著啊", "妈妈护他娘的新亚非多点第二个派人岩春信用社", "金成", "--", "didn't were", "u"] ["做方案昨天说句下午市场旅之星他娘的逝去", "学员", "mean", "叁万", "or"] [] ["但是文宁暂时一级下周专业备份样品语音", "", "?"] ["p", "x", "m", "q", "g", "e", "x", "p", "e", "x"] ["x", "v", "q", "t", "b", "s", "r", "u"] ["j", "a", "y", "n", "k"] ["s", "h", "e", "w", "y", "j", "y", "b", "k", "c"] \N [2027-01-09, 2023-12-17, 2023-12-17, 2026-01-18, 2023-12-14] [2024-01-08, 2023-12-16, 2024-02-18, 9999-12-31, 2023-01-15] [] [2026-01-18 00:00:00, 2027-01-16 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02, 2027-01-16 00:00:00, 2023-12-10 00:00:00] [9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2023-12-14 00:00:00] [2023-12-14 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00, 2023-12-12 00:00:00] [] [2023-12-10 00:00:00, 2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2009-02-16 00:00:00, 2025-06-18 00:00:00] [] [] [2023-12-13 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00] [] [] [9999-12-31 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2000-04-10 00:00:00, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00] +150 467908960 1061461317 330440296 -1 2023-01-15 2023-12-12 2023-12-12 2026-02-18 ? one out say one have oh no if okay was got if there 家庭 铭岳不应该一两做不到金牌经销总代理魏军李总阿里巴巴申请人身份证 -- - 世安 \N 1 100.020000 40.123000 1.200000 24.047500 2024-02-18 00:00:00 2024-06-30 12:01:02.123000 2023-12-18 00:00:00 2024-01-09 00:00:00 2023-12-20 00:00:00 2023-12-09 00:00:00 2027-01-09 00:00:00 2024-01-08 00:00:00 [0, 1, 0, 0, 0] \N [1, 1, 0, 1, 1] [0, 0, 0, 0, 0, 0, 1, 0] [-1, -81, -128, 1, -65] [26, 127, -128, 1, -30, -101, 1, 8] [-107, -128, 66] [17, 86, -1, -128, 68, 92, 43, -1] [1, 30659, -1, 12375, -14481, 25092, 1, -1] [24101, -8534, -32768, -1, -24689, 0, -24392, -20488, 1, -31234] [-12350, 0, -16178] [12806, 0, -21196, 30424, 0, 32767, -3442, 2504] [32679, 0, 2147483647] [260479664, 1016392081, 0, 2147483647, 1786608812] [-1288498908, -185732626, 32679] [-222540056, -1450718166, -1250210305, 31088640, 0] [-7024831870634261039, 6039651130206717851, 0, -1361436776, -9223372036854775808] [9223372036854775807, -1, 245, 147483648, 1046154320, 1, -9223372036854775808, 32679] [0, 162684365, 9223372036854775807, 147483648, -1898864578] [5537050394095839219, 1457157502, 9223372036854775807, -1, -578376104616999772, 4555118, 7916926567517341496, -983837167, -2742853, 32679] [-329489925, -2, -2] [-2, -2, -1] [] [-2, -1, 0, -1, 1, 0, 1, 0, 8, 0] [40.123000000000, 52.162200000000, 300.343000000000, -1.200000000000, 39.000000000000, 100.020000000000, 100.020000000000, 0.000000000000] \N [30.051800000000, 47.059400000000, 100.020000000000] [13.087400000000, 7.195700000000, 300.343000000000] [] [3.0415, -1.2000, 300.3430, 40.1230, 1.2000, 40.1230, 100.0200, 1.2000] [] [19.0163, 40.1230, 1.2000, 25.0998, 1.2000, 0.0000, 93.1841, 39.1716, 0.0000, 100.0200] [99, 40, 56, 0, 40, -1, 100, -1, 1, 300] [] [-1, 72, 27] [100, 40, -1, 0, -1] ["", "这三大", "--", "扩展名普腾凄楚你不", "弹出", "--", "-", "they look then he's some he's oh was"] [] ["上网高速补上费心也不是去做金牌本来才好讯息", "u", "--", "I'm", "降低刘汉", "didn't", "y", "?"] ["太原分辨率孙健要是几下长远开票往年", "-", "这首", "get", "--"] ["that's his in", "那份", "f", "看中的", "just"] ["at", "人数", "蓝色", "you're", "tell mean yes out", "?", "be yeah mean get how had her get your because", "", "it's", "新鑫"] ["何时硅谷类型性格", "记录鼠标课酬既然大姐龙成系统", "任务中铁行政世上王海之日起扩展名机芯就会"] ["q", "up come he or were can", "at", "--", "?", "扬子", "跑跑卡丁车旅之星这几太难凡是", "no"] [] \N ["j", "z", "b", "y", "h", "s", "l", "s", "w", "z"] ["u", "h", "c", "l", "j", "u", "j", "t", "i", "q"] [2026-01-18, 2023-12-17, 2024-01-09, 2024-02-18, 2023-12-11] [] [2026-01-18, 2023-12-14, 2025-06-18, 2023-12-13, 2023-12-16, 2024-01-19, 2025-06-18, 2023-12-15, 2024-06-30, 2023-12-18] [9999-12-31, 2024-01-19, 2018-11-28, 2027-01-09, 2023-12-15] [9999-12-31 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00, 2013-11-24 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59, 2024-06-30 12:01:02] [2023-12-19 00:00:00, 2023-12-12 00:00:00, 2023-01-15 08:32:59, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2019-03-04 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2014-08-12 00:00:00, 2014-08-12 00:00:00] [2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-12-20 00:00:00] [2023-12-20 00:00:00, 2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2023-12-12 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2024-01-08 00:00:00, 2025-02-18 00:00:00] [2025-06-18 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00] [2024-07-01 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00] [2023-12-11 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] [2023-12-18 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00] [2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00] +151 147483648 -1909366138 -2147483648 852860036 2023-12-14 2024-06-30 2025-02-17 2024-02-18 it think can the right could as really some some get don't 群主胶片厂说吧通道不少承担遮盖 d she you're are out think 1 0 100.020000 1.200000 48.155300 12.029600 2023-12-20 00:00:00 2008-03-01 00:00:00 2024-01-19 00:00:00 2024-02-18 00:00:00 2023-12-20 00:00:00 2023-12-09 00:00:00 2023-12-11 00:00:00 2023-12-11 00:00:00 [1, 0, 1, 0, 1] [0, 1, 1, 0, 0, 0, 0, 1, 1, 1] [] [1, 1, 1, 1, 1, 1, 0, 1, 0, 0] [-43, 27, 0, 0, 1, 87, 22, -1, 0, -1] [-128, 126, -80, -1, 127, 110, 5, 1, 1, -84] [] [65, 80, 1, -128, 9, 1, -50, -1, 1, 33] [32767, 32767, 13632, 32767, 24284, 17939, 8458, 245] [32767, 32767, -13309, 32767, -32768] [32767, -32768, 32767] [] [-709918974, 0, 32679, -1, -999958315, 2147483647, 1035539929, -2147483648] [147483648, 1468458406, -449360230, 32679, 0, -535195893, 147483648, 32679] [-2147483648, -995481568, 147483648, -422234752, 3869003, 1704741439, -586026809, -1, -2075726304, 1999614597] [32679, -1389983209, 1959808254] [] [-9223372036854775808, 0, 1, -9223372036854775808, -6879400330032451406, -1141497955, -1, -2926792241991717470] [1, 3373455972026788118, -9223372036854775808, -1, 9223372036854775807, -6900941834983405658, 1, -1245122919627761255] [245, 245, 147483648, 147483648, 5830564912916351550, -9223372036854775808, 4399361232721555240, 1] [0, 0, -437635516, 0, -1297572303, 1, -2, 1] [8, 2, 1, -1, -2, 1, 0, -2, 1, 9] [-1, 0, 356660955, -1, -2] [374911164, -1, 0, 0, -1, 9, 74416554, 1, -2, -591148051] [] [0.000000000000, 300.343000000000, -1.200000000000, 1.200000000000, 89.073700000000] [100.020000000000, -1.200000000000, 21.144000000000, 98.091900000000, 100.020000000000, 65.105400000000, 300.343000000000, 1.200000000000, 0.000000000000, 32.112600000000] [40.123000000000, 100.020000000000, -1.200000000000, -1.200000000000, 100.020000000000, 40.123000000000, 44.155000000000, 37.041500000000, 85.146600000000, 40.123000000000] [14.0378, 37.0222, -1.2000, 0.0000, 1.2000, 300.3430, 37.0610, 300.3430, 50.1614, 1.2000] [1.2000, 46.1491, 53.1075, 40.1230, 68.0982] [1.2000, 0.0000, 40.1230] [1.2000, 0.0000, 58.0981, 100.0200, 300.3430, 1.2000, 97.0904, 40.1230, 16.0824, 100.0200] [-1, 100, -1, -1, 18] [53, 100, -1, 0, 100, 59, 51, -1, 300, 300] [300, 300, 100, 1, 300, 85, 98, 1] [300, 1, -1, 20, -1] ["?", "生病景泰性公司人参阿里巴巴知道说过严重", "had"] ["天网邮寄等级耽误总也吃过饭无所谓列举开封", "湖边赵总好人坏人难过曙光凌峰", "p", "问起", "why", "?", "is", "your going that's right not there", "?", "下个礼拜"] [] ["c", "but", "that's you would why no", "有个", "got time don't okay", "最后智者看不懂输入游民", "聂柯", "--", "him", "行货服务器"] ["感触", "ok from a", "走在出去这一生压力无限天翔认可科汇", "则很难造成单核液晶问过", "can like just and"] ["?", "-", "讨论组"] ["q", "一款等着摄像头订货", "神偷"] [] ["z", "z", "d", "u", "b"] ["j", "u", "z"] ["n", "w", "a", "s", "n"] ["c", "q", "l"] [2025-06-18, 2024-02-18, 2024-02-18, 2023-12-19, 2024-02-18, 2023-12-13, 2024-06-30, 2024-01-17] [2024-08-03, 2024-07-01, 2023-12-16] [2023-12-11, 2014-08-12, 2023-01-15, 2023-12-19, 2024-02-18] [2023-12-09, 2027-01-09, 2027-01-16, 2024-02-18, 2023-12-20, 2014-08-12, 2024-02-18, 2027-01-09] [2014-08-05 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2023-12-19 00:00:00, 2024-07-01 00:00:00] \N [] [2025-02-17 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-11 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2017-09-04 00:00:00, 2023-12-16 00:00:00] [2027-01-09 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123000] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-07-01 00:00:00, 9999-12-31 23:59:59, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00] [9999-12-31 23:59:59, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2024-07-01 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2024-01-17 00:00:00] [2023-12-12 00:00:00, 2008-03-09 00:00:00, 2024-02-18 00:00:00] [] +152 1065672055 32679 -1664406586 1960449447 2024-02-18 2027-01-09 2023-12-11 2026-01-18 到时 a 孤独你给发送服装五笔调价一个标注孙健 商都 think back the 文华谦虚对待搞定 自学课题讨论组 - \N 1 40.123000 -1.200000 300.343000 100.020000 2024-08-03 13:08:30 2024-01-19 00:00:00 2023-12-09 00:00:00 2024-01-17 00:00:00 9999-12-31 23:59:59 2025-02-18 00:00:00 2026-01-18 00:00:00 2023-12-17 00:00:00 [1, 1, 1, 1, 0, 1, 1, 0] [0, 0, 1] [] [0, 0, 1] \N [] [15, 127, 0, 1, 7, 20, 0, 0, 1, -128] [1, 66, 127, 75, 91, 0, 127, -1, -27, -112] \N [0, -11589, 1, -19173, 28234, 1, 32767, -32768] [] [32767, 482, -20239, 1, -6137, 0, 0, -16318, -1, 29063] [-1, 500282449, 32679, -87550716, 2147483647] [-800165135, -1, 1893442325, -2147483648, -1620166411] [] [1, 1823842606, 92078793] [-2021108665, 8325498638470450919, -9223372036854775808] [0, 245, 245] [9223372036854775807, 2089729162262816791, -9223372036854775808, 621130290342813847, 0, 147483648, 1416536268, 1] [147483648, -699382454, 8761200693136696396] [0, -1, 1892467792] \N [-2, -1, -1] [-1, 1, 0, -2, -2] [-1.200000000000, 300.343000000000, 1.070800000000, 69.083300000000, 40.123000000000, 1.200000000000, 100.020000000000, 23.035700000000, -1.200000000000, 38.059200000000] [77.178200000000, 80.004100000000, 1.200000000000, 0.000000000000, 100.020000000000] [66.121200000000, -1.200000000000, 35.163200000000] [1.200000000000, 40.100900000000, 32.149800000000, 1.200000000000, -1.200000000000, 300.343000000000, 0.000000000000, 59.187600000000, 100.020000000000, 98.047300000000] [100.0200, 0.0000, -1.2000, 300.3430, 40.1230] [40.1230, 0.0000, 82.0577, 1.2000, 40.1230, 1.2000, 40.1230, -1.2000] [56.0835, -1.2000, 44.0701, 0.0000, 99.1732, 1.2000, -1.2000, 100.0200, 1.2000, 81.1927] [] [1, 0, -1] \N [40, 0, 40] [] ["but", "一周要时间不已元月低配", "对话框硬盘爱国心有成交领出圣龙在你是", "铭岳关键收尾名字", "关于对方", "发过南天河南郑州舞阳", "-", ""] ["-", "长期", "通话", "已经威达", "再来"] ["", "w", "are", "所要求祝福干干净净电话给我", "--"] ["?", "--", "?", "东海对策陪伴小人回复大厦", "他在哪", "--", "隆康位置华康花钱蓝奥", "吉林公寓伴我到南村心事"] ["记得聪明引进过来几天跟单黄委会对方电粉尽管", "潇湘奸商冰火", "", "my your all here", "going well then he so", "", "can", "梅捷", "效果参与", "兰州宋玉慧岩春"] \N ["just hey then not if", "", "?", "--", "-", "--", "焦煤", "-", "--", "--"] [] ["w", "n", "t", "v", "n"] \N ["y", "r", "o", "p", "q", "v", "t", "a", "x", "b"] ["a", "m", "d", "d", "j", "n", "p", "z", "f", "u"] [2023-12-18, 9999-12-31, 2024-07-01, 2024-01-17, 2023-12-19, 2023-12-18, 2024-06-30, 2023-12-17, 2024-01-17, 2027-01-09] [] [2023-12-09, 2024-01-31, 2026-01-18, 2024-02-18, 2027-01-16, 2023-12-13, 2023-12-14, 2024-08-03] [2027-01-09, 2024-06-30, 2023-12-16, 2023-12-15, 2026-01-18] [2024-08-03 13:08:30, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00] \N [2023-12-11 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00] [] [2023-12-13 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00] [2026-01-18 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2019-09-23 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-18 00:00:00] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00] [2023-12-10 00:00:00, 2024-01-17 00:00:00, 2016-06-10 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00] [2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2024-07-01 00:00:00] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00] [2025-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00] +155 147483648 -727320476 -531843899 -1648874242 2027-01-09 2024-06-30 2023-12-11 2024-01-19 拿不到七匹狼一些元超其实联动这种 we can't have oh why for can 高科 -- -- 初五几万权威开炮博源产品事业部经理访问量张丽萍 ok then 0 0 100.020000 0.000000 0.000000 -1.200000 2024-01-09 00:00:00 2024-02-18 00:00:00 2025-06-18 00:00:00 2024-01-31 00:00:00 2027-01-16 00:00:00 9999-12-31 23:59:59 2023-01-15 08:32:59.123123 2026-01-18 00:00:00 [0, 1, 1, 0, 0, 0, 1, 0] [0, 0, 0, 1, 0] [1, 1, 1, 1, 1, 0, 0, 0, 1, 1] [] [-128, -25, -43] [3, -128, 81, 127, 127, -1, -3, -1] [63, -128, -128, 1, -1] [127, 127, -102, 0, -128, 1, 0, 58] [32767, -1, 1, 22057, -17303, -1, 27754, 32767, -2215, 0] [1, -31785, 31052] [-9104, -25169, -29753] [] [] [1327370097, 1135796413, 659126838] [491053643, -1913785235, 306755404, 2147483647, -1260308253, -1521473723, 0, -618292300] [] [5122189759083521942, 1, 1742093, -9223372036854775808, 884483558, -1, 287400819395893368, 1, 80093115121156311, 0] [1, 245, 32679, -9223372036854775808, -9223372036854775808] [245, -3321155, -2386988718624938045, 245, 1] [-1, -8142992, 9223372036854775807, 1, 1] [-1410627325, -2068605924, -1417196485] [-2, 79801999, 1, 6, -825118733] [] [] [13.145600000000, 30.033700000000, 0.000000000000, 1.200000000000, 100.020000000000, 0.000000000000, 81.052700000000, 4.018600000000, 21.069800000000, 100.020000000000] [100.020000000000, -1.200000000000, 1.200000000000, 61.024100000000, 100.020000000000, 56.008800000000, 40.123000000000, 1.200000000000] [100.020000000000, 0.061300000000, 1.200000000000, 100.020000000000, 69.147900000000, 1.200000000000, 64.056500000000, -1.200000000000, 100.105100000000, 0.000000000000] [-1.200000000000, 300.343000000000, 100.020000000000] [40.1230, 40.1230, 0.0000, 17.1497, 41.0667, 40.1230, 40.1230, 40.1230] [] [] [0.0000, 100.0200, 0.0000, 300.3430, 85.1192] [14, 40, 40, 83, 1, 100, 44, 91] [] [1, 16, 28, 0, 0, 1, 81, 36, 300, 1] [19, 60, 0, 40, 1, 29, 100, 65] ["when is don't her", "盛唐黄经理改动尼奇几十年新闻", "if", "is this him now", "not", "含义", "-", "can't if so some up"] \N ["?", "-", "now", "嵩县", "验货单", "all could yeah", "--", "晚上", "like", "--"] ["--", "太厚", "-", "", "", "大概谈谈气死疲劳水利公司磁盘柜第三方", "金立但愿漫长治好一开始我到性能", "飞鸽", "--", "--"] ["--", "探明", "--", "to", "讲的儿孙们同力有人那边哪里采用天孝"] ["聪明文艺艺术进入了偷出原著啊开发我一会给大几个", "there as", "--", "--", "-"] [] ["are", "想不到", "e", "检验有机会提到货晨晖现货部队培新", "打印机海民纱卡个哦这部分输入展开大会", "do that", "联系一览表不想价值倒是武侠组哟半天班屏蔽飞达气人", "冰火", "-", "--"] \N ["j", "c", "t", "g", "q"] ["k", "k", "r"] ["c", "p", "a", "t", "s", "t", "a", "a", "c", "j"] [2023-12-16, 2024-01-19, 2023-12-20, 2023-12-10, 2024-06-30] [2026-02-18, 2025-02-18, 9999-12-31, 2027-01-09, 2025-02-18, 2024-06-30, 2023-12-11, 2023-12-18] [2023-12-09, 2024-07-01, 2024-06-30, 2023-12-19, 2025-02-18, 2023-12-14, 2023-01-15, 2023-12-12, 9999-12-31, 2011-02-25] [2023-12-16, 2023-12-09, 2027-01-09, 2014-08-12, 2024-01-31, 2024-01-19, 2025-02-17, 2024-02-18] [2023-12-11 00:00:00, 2023-12-19 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00] [2024-01-19 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00, 2011-03-14 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2024-02-18 00:00:00, 2011-11-20 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00] [2023-12-17 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 9999-12-31 23:59:59, 2024-06-30 12:01:02, 2024-01-09 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30] [2025-06-18 00:00:00, 2027-01-16 00:00:00, 2013-12-03 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30] [2014-08-12 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00] [] [2024-06-30 12:01:02.123000, 2024-01-08 00:00:00, 2023-12-17 00:00:00] [2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2024-07-01 00:00:00, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00] [] [9999-12-31 23:59:59, 2023-12-19 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00] [2023-12-18 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00] +156 -1674423614 0 190791607 1162584702 2025-02-17 2023-12-09 2024-01-09 2024-02-18 there 余下 \N 位置 w 尽管电子金鼎好久不见结婚发给 n - 1 0 \N 46.125900 100.020000 40.123000 2024-01-08 00:00:00 2026-02-18 00:00:00 2023-12-17 00:00:00 2023-12-15 00:00:00 2027-01-09 00:00:00 2023-12-13 00:00:00 9999-12-31 00:00:00 2014-08-12 00:00:00 [0, 0, 0] [0, 0, 0, 1, 0, 0, 1, 1, 1, 0] [] [0, 0, 0, 1, 0, 0, 0, 0] [-128, 88, -128] [-1, -128, 1, 1, 60] [-48, 0, -75, 56, 92, 1, 0, 123, 0, 69] [25, 54, 1, 92, 1, 0, -128, 113] [11773, 1, -10803, -23183, -24746] [-11397, -32768, 0, -3837, 21219, -32768, 13012, 22738] [-10775, 18214, 20323, 20580, -18887] [30749, -2377, 0, 0, -32768] [2044268623, -1763277472, -1543781631, 1, 147483648, 1, -1567697678, 32679, 2024740409, 1391509254] [2147483647, 1908312336, -2147483648, 2147483647, -1180254277, 32679, -1480819278, 1, -742062638, 359596871] [] [138663669, 1715266050, 982698113, -230096179, 147483648] [6746007623249543335, 147483648, 1401013535118018921, 1, -656257541] \N [9223372036854775807, 147483648, -1, 1787138931223426393, 0, -4053579439584365888, -9223372036854775808, 1] [] [-2, -2, -803998280, 5, -1, 1631551363, 6, 1584250465, -1, -1] [] [2018286362, -1, 1, -2, 2] [] [300.343000000000, -1.200000000000, 73.140500000000, 17.161100000000, 0.000000000000, 63.147700000000, 56.059700000000, 300.343000000000] [1.200000000000, 300.343000000000, -1.200000000000] [] [40.123000000000, 1.200000000000, 1.200000000000] [40.1230, 30.0219, 100.0200] [300.3430, 68.1460, 40.1230] [300.3430, 31.1567, 100.0200, 100.0200, 40.1230] [100.0200, 300.3430, 68.1447, 44.0473, 96.0486] [] [91, 40, 0, 100, 54, 40, 54, 30] [60, 22, 100] [34, 76, 1, 1, 100, 300, 0, -1] ["?", "t", "just", "v", "w", "right", "get a the", "like"] ["mean have", "a", "政策这几天"] ["腾飞华中又去中龙宝龙", "f", ""] [] ["?", "--", "on", "-", "--", "天讯河南瑞达五级可以治疗那售后第三方年前智慧", "集成挑战没错夜间世上清楚", "can"] ["on", "his why he had up", "--", "don't", "something how but here with time", "?", "-", "j"] [] [] ["w", "c", "s"] [] ["d", "z", "i", "k", "s", "r", "y", "q"] ["x", "j", "k"] [2027-01-09, 2023-12-14, 2025-02-17, 2014-08-12, 2023-12-12, 2024-06-30, 2003-10-21, 2023-12-09] [2023-12-20, 2023-12-11, 2023-12-19, 2023-12-19, 9999-12-31, 2025-02-17, 2024-02-18, 2027-01-09, 2025-06-18, 2023-01-15] [2023-12-14, 2024-02-18, 2023-12-13, 9999-12-31, 2016-05-28] [2024-02-18, 2023-12-19, 2024-01-31, 2024-08-03, 2027-01-09] [2026-01-18 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-07-01 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2004-12-14 00:00:00] [2024-08-03 13:08:30, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-10 00:00:00] [2024-01-09 00:00:00, 2025-02-18 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02, 2025-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00] [] [2024-07-01 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2011-01-07 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00] [2027-01-09 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00] [2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-07-01 00:00:00, 2026-02-18 00:00:00, 2003-05-20 00:00:00, 2024-02-18 00:00:00, 2015-03-26 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-15 00:00:00, 2023-12-14 00:00:00] [] \N [2027-01-16 00:00:00, 2024-01-19 00:00:00, 2001-11-21 00:00:00] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00] [2008-08-06 00:00:00, 2024-01-17 00:00:00, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2023-12-10 00:00:00, 2023-12-15 00:00:00, 2024-07-01 00:00:00, 9999-12-31 00:00:00] +157 638867322 1 1935876920 -1672317911 2024-07-01 2018-05-27 2023-12-12 9999-12-31 okay -- - 恢复工业大学地区鸡巴 线上佳成易插拔孙娟这上面王总签不签都容量小孩 how t 0 1 300.343000 19.056900 300.343000 300.343000 2023-12-17 00:00:00 2023-12-20 00:00:00 2024-01-09 00:00:00 2024-07-01 00:00:00 2023-12-19 00:00:00 2025-06-18 00:00:00 2024-01-08 00:00:00 9999-12-31 00:00:00 [] [0, 0, 0, 0, 0, 0, 0, 0] [1, 0, 0, 0, 0, 0, 1, 1] [0, 1, 0, 1, 0, 0, 0, 0, 0, 0] \N [127, 54, -1] [115, -128, 0, 1, -1] [-128, -126, -39] [] [5567, 0, -1, 245, 1918, 9924, 21299, -6436, -1892, -32768] [0, 22152, -25019] [5211, 245, 245] [0, -2147483648, 1, 32679, 2147483647] [] [] [] [-1, 950675160175371908, 4054694795846295443, 32679, 1] [147483648, 32679, 245, -1450249871, 5225606878346202707] [4908746772261975388, 147483648, 245, 1688085379, -9065185656657744370, 1, 245, 1] [-9223372036854775808, -9223372036854775808, 9223372036854775807, 245, 147483648, -1, 1, 1742214543694781103] \N [8, -1, -2, 0, -1349750609, -1, -2, -1] [-505424205, -1, -1732477618] [1, 9, 0, 0, -1, 1, -1, 1] [40.077800000000, 0.000000000000, 40.123000000000] [52.040100000000, 66.116600000000, 300.343000000000, 45.131400000000, 0.000000000000, 100.020000000000, -1.200000000000, 0.000000000000, 40.123000000000, 79.086900000000] [-1.200000000000, 0.000000000000, 300.343000000000] [66.001300000000, 78.049500000000, 0.000000000000] [69.0236, 40.1230, 50.0103, 100.0200, 11.1699] \N [51.1986, 100.0200, 1.2000, 300.3430, 1.2000, 0.0000, 0.0000, 7.0673] [1.2000, 22.0705, 1.2000, 0.0000, 300.3430, 65.0940, 40.1230, 0.0000] [300, 3, 40] [-1, 83, 24, 100, 83] [0, -1, 70] [] ["would now no i if here would", "hey then were going", "--", "?", "he's", "", "in", "零捌是吗壹万说出来分销环保局还有相见"] \N ["-", "-", "I'm", "why", "彩虹", "代码三种欧洲圣龙邮政不需要换一家和兴不对外", "in", "q"] ["中毒", "帮帮", "?"] ["got want", "?", "j", "m", "", "t", "-", "did is be he look could they got are"] ["yeah didn't can do for go was go", "s", "科技", "s", "预防", "--", "赵经理最大化刘亚五万条飞信工程师轻松配件能进收条", "w", "get going to there know hey can are", "could why didn't what this one a"] ["数目重新一年中记下房间", "--", "一贯", "平常", "在你是", "is good tell we ok good yeah that but", "tell him he's not with he have say hey the", "good", "it her something who tell okay now mean her", "二姑"] ["so is you're to for it would i it look", "原则认为回答问题废话付款实际上俩年人家端口闺女", "--", "", "个哦交到贰仟直接车费西开发收吧交换机石龙愉快", "?", "五万", "之后", "about", "彼此倪玲梅春子熟人索尼儿孙们高科吃饭了"] ["s", "c", "z"] ["i", "i", "t"] ["c", "n", "k", "y", "w", "k", "t", "h"] ["a", "y", "a"] [] [2000-03-27, 2024-02-18, 2023-01-15] [] [2023-12-19, 2026-02-18, 2023-12-10] [2024-06-30 12:01:02, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00] [2026-01-18 00:00:00, 2023-12-15 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00] [] [2023-12-11 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2024-01-31 00:00:00] [2009-04-26 00:00:00, 2023-12-16 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 9999-12-31 00:00:00] \N [] [2024-07-01 00:00:00, 9999-12-31 00:00:00, 2025-02-17 00:00:00, 2023-12-17 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00, 2024-01-09 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00] [2023-12-15 00:00:00, 2023-01-15 08:32:59.123123, 2027-01-09 00:00:00] [9999-12-31 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00] [2023-12-14 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00, 2023-12-10 00:00:00] +158 32679 -2147483648 -780361366 -2147483648 2027-01-09 2023-12-19 2023-12-15 2025-02-17 be hey as his he's been have 睡觉来访你是哪位黄河科学系统集成商不确定课题这场圪当店乡划分 ? 正式版保护来志光创彩万一短句 老了 -- there \N 1 300.343000 -1.200000 76.132400 36.060300 2024-06-30 12:01:02.123000 2023-12-15 00:00:00 2023-12-19 00:00:00 2024-01-08 00:00:00 2027-01-16 00:00:00 2024-06-30 12:01:02.123000 2025-06-18 00:00:00 2023-12-16 00:00:00 [] [0, 0, 0, 1, 1, 0, 1, 0, 0, 1] [1, 1, 0, 0, 0, 1, 1, 0, 0, 1] [1, 0, 1] [1, 0, -128, -31, 0, 1, -76, -114, 8, -81] [25, -32, 127, 104, 1, -1, 0, -23] [95, -128, -34, 1, -2, 127, 6, 0] [9, -128, 1, 1, 1] [-1, -23476, 32767] [-31559, -2731, 0, 15258, 0] [245, 245, -31159] [1, 32767, 10078, 13922, 245, -32768, -16650, -32768, 15195, -1] [147483648, 1, 32679, 1, -59337637, -1, 0, -2147483648, 0, 2147483647] [147483648, 32679, 1916311291, -932990416, -2147483648] [] [2147483647, 1, -2147483648, 862041655, 1, -104140507, -434809471, 32679] [4161091912749216606, 6922547117084862124, 32679, -3588666867163629919, 245] [245, 32679, 32679, -1784618957128361744, 9223372036854775807] [245, -9223372036854775808, -1] [-9223372036854775808, -4028530428051420943, -9223372036854775808, 1, -5841370363355317735, 32679, -1, 5585652906438402830, 1961953061, 245] \N [-2, -2, -1, 1, -1] [0, 5, 679561760, 4, 1, 0, 1, 0] [418811325, -1, -2] [] [40.123000000000, 15.022900000000, 100.020000000000] [] [1.200000000000, 17.000400000000, 1.200000000000] [-1.2000, 92.0933, 1.2000, 300.3430, 1.2000, 19.0449, 0.0000, 40.1230] [] [40.1230, 300.3430, -1.2000, 100.0200, 0.0000, 300.3430, 1.2000, 1.2000] [-1.2000, 0.0000, -1.2000] [100, -1, 5, 100, 300, 26, 16, 300] [-1, 3, 100, 95, 40, 300, -1, 40, 0, 300] [1, 56, 19, 0, 300, 1, 0, 48, 1, 100] [100, 300, 40, 47, 0, 100, 0, 300, 77, 100] ["一开始", "?", "手机", "her but want have there", "证明最上面天下便宜优先用在东京不对业务", "状态", "", "it"] [] ["", "v", "--", "?", "had why this some not your going did", "利盟", "--", "", "from been", "这个"] ["?", "d", "中龙", "", "mean yeah", "go", "翻译梅捷分享一次初八", "我刚来呀出团天数单核吵架数码打我也是焦作"] ["don't", "did look", "-", "I'll", "好不好王总签不签都普庆真正以以上的", "tell out didn't here so not are would when what", "一点若虚唱给资料", "how and of", "夜市有空春节一切", "则会有"] \N [] [] [] [] ["u", "n", "v", "j", "y"] ["v", "f", "g", "b", "r", "c", "b", "c", "j", "h"] [2023-12-18, 2027-01-09, 2023-12-20, 2025-02-18, 2014-08-12, 2026-01-18, 2023-12-12, 2023-12-13] [2023-12-13, 2024-06-30, 2024-02-18, 2027-01-09, 2026-01-18, 2023-12-15, 2026-02-18, 9999-12-31] [] [2023-12-19, 2023-12-10, 2024-01-17] [9999-12-31 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2001-10-16 00:00:00] \N [] [2026-02-18 00:00:00, 2014-08-12 00:00:00, 2008-12-27 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00] [2024-08-03 13:08:30, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00] [2023-12-15 00:00:00, 2024-01-17 00:00:00, 2026-01-18 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [2024-01-19 00:00:00, 2023-12-09 00:00:00, 2023-12-09 00:00:00] [2026-02-18 00:00:00, 2023-12-11 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00] [2024-07-01 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-06-30 12:01:02.123000] [2023-01-15 08:32:59.123123, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2023-12-10 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00, 2015-06-05 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00] [2023-12-16 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 23:59:59, 2023-12-14 00:00:00, 2025-02-17 00:00:00] [] +159 -1 32679 2147483647 1368180864 2023-12-17 2024-07-01 2027-01-16 2025-02-17 y 报表 did \N 帮不了你 呀库存 花钱不方便纸业感情请到 tell no got what were and you're 0 1 40.123000 54.150300 100.020000 20.173800 2026-02-18 00:00:00 2024-01-17 00:00:00 2024-07-01 00:00:00 2026-01-18 00:00:00 2016-03-05 00:00:00 2024-07-01 00:00:00 2026-02-18 00:00:00 2024-08-03 13:08:30 [1, 1, 0, 0, 0] [1, 1, 0] [0, 1, 1, 0, 1] [0, 1, 1, 0, 1, 0, 1, 0] [-1, 1, 8, -1, -128, -63, 0, -125, -128, 127] [] [-1, 0, 0, 1, -128] [1, 127, -5] [-13485, -1, -6040, -32768, -14297, -32768, 16392, 245, -1, 32767] [27459, 1966, 26299, -28260, -9360, 2798, 18854, 245, 26160, 245] [] [27884, 29292, 32767] [-923403655, 0, 1, -2095305304, -2147483648, 160907298, 729616466, 1878171669, 2147483647, 0] [922831486, -1820953961, -1, 2147483647, 0, 1948388036, -2147483648, 1399095962] [32679, 32679, -2147483648] [276216036, 155278955, 2147483647, 1, 147483648, -1789506811, 1, -2147483648, -2147483648, 1899768425] [147483648, -9223372036854775808, -3625959, 147483648, 0] [] [] [32679, 147483648, -1, -9223372036854775808, 1, 6597390379157157670, -6476626433717647736, 6979400207537984834] [-1168435405, -2, 9, 1, 1420075424, -2, 7, 657425804, 329366598, -1] [6, -2, 8, 5, 0, 3, 0, 1, -1, 6] [-2, 0, -2] [1, -1878826661, 0, 0, -1776639537, 6, 0, -1, 0, -2] [57.014500000000, 79.018000000000, 100.056900000000] [25.132300000000, -1.200000000000, 6.158300000000, 95.074300000000, 1.200000000000] [1.200000000000, 10.045800000000, 300.343000000000, 10.084400000000, 77.039100000000] [100.020000000000, 1.200000000000, 0.000000000000] \N [62.0854, 0.0000, 1.2000] [100.0200, 0.0000, 100.0200, 1.2000, 5.0689, 100.0200, 8.0182, 100.0200, 50.0430, -1.2000] [3.0930, 0.0000, 47.1060, 100.0200, 62.1472, 0.0000, -1.2000, 100.0200] [40, 40, 300] [300, 300, 40, 40, 300, 100, 21, 88] [] [40, 40, 40] ["my this see", "总代理安全主奴婢总经理", "can't back say know when okay", "唯一改掉格飞", "her", "--", "", "百货", "who", "扩展板"] ["?", "-", "to"] ["mean", "旭日", "艾蕊", "", ""] [] ["", "you're", "--"] ["her", "?", "o"] [] ["舍得", "", "才是商都稳定瑞达确认海域哦轻松成本"] ["c", "z", "t", "t", "p"] ["l", "q", "l"] ["q", "s", "v", "e", "x", "u", "k", "l"] ["s", "v", "j"] [2024-07-01, 2023-12-09, 2027-01-09] [2026-01-18, 2023-01-15, 2024-07-01, 2023-12-10, 9999-12-31, 2026-01-18, 2024-01-08, 2026-02-18, 9999-12-31, 2024-01-08] [2023-12-10, 2026-01-18, 2023-12-19, 2026-01-18, 2024-01-08, 2027-01-09, 2024-02-18, 2024-08-03] [] [2023-12-14 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00] [2023-12-13 00:00:00, 2008-04-15 00:00:00, 2023-12-11 00:00:00] [2015-09-25 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00] [2026-01-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00] \N [2023-12-19 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2014-08-12 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59] [] [2024-06-30 12:01:02.123000, 2023-12-14 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [] [2023-12-13 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-12 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00] [] +160 -2147483648 \N 0 749882292 2023-12-18 2025-02-18 2023-12-14 2025-02-17 they about was something because 光通路诊断一下阿调价一小时 -- here one ok could good see see 不大一会吧 right can on you did o 0 0 100.020000 0.000000 -1.200000 300.343000 2023-12-12 00:00:00 2023-12-12 00:00:00 2023-12-10 00:00:00 2024-01-17 00:00:00 2023-12-19 00:00:00 2023-12-19 00:00:00 2024-07-01 00:00:00 2024-02-18 00:00:00 \N [0, 0, 0, 0, 0, 1, 0, 0] [0, 0, 1, 0, 0] [0, 1, 0] [-128, -128, -1, -89, 127] [1, -128, -1, 3, -67] [127, 63, 127, 38, 127] [127, -128, -106, -128, 127, 112, 127, 0] [-22889, -2288, 32767, -10688, 18135, 0, 245, 18411] [28552, 26083, -32768] [1, 32767, -9057, -23875, -11741] [-32768, -15120, -22495, -702, 4631, -28813, 32767, 28886, 24284, -32768] [-1, -1, 1226521750, -1677516318, -1634124501, -1469646532, 31338468, 147483648, -1926453701, 971758937] [-640268013, 926675487, 0] [1, 147483648, -669307951, 2147483647, 864466017] [] [-9223372036854775808, -1, 32679, 0, 4478857467515621023] [] [147483648, -2513323282509861424, 0, -1, 6589937223920148743, 147483648, 1, 9223372036854775807, -9223372036854775808, -4230663087518422833] [] [-1, -2, 1, 0, -1, -1, -2, 1] [-1, -2, 0, -862273790, 1, 7, -2, 3] [1211776208, -2, -2, 2083954598, 1] [1, 1, 1] [40.123000000000, 22.019100000000, 40.123000000000, -1.200000000000, 77.159500000000, 0.000000000000, 98.020500000000, 19.033500000000, -1.200000000000, 1.200000000000] [1.200000000000, 100.020000000000, 100.020000000000, 45.193000000000, -1.200000000000] [0.000000000000, 49.058800000000, 300.343000000000, 22.152200000000, 300.343000000000, 0.000000000000, 3.174400000000, 25.066000000000, 0.000000000000, 1.200000000000] [40.123000000000, 1.200000000000, 1.200000000000] [24.0204, 35.1879, 0.0000, -1.2000, 1.2000, 6.1105, 22.0615, 40.1230] [300.3430, -1.2000, -1.2000, 35.0966, 40.1230] [1.2000, 40.1230, 40.1230, 300.3430, 1.2000, 40.1230, 73.1393, 58.1409] [47.1722, 300.3430, 40.1230, 40.1230, 90.0418, 40.0232, 23.1731, 42.0531] [40, 58, 1] [-1, 0, -1, -1, -1] [100, 100, 100, 93, 1, 100, 1, 78, -1, 0] [0, 100, 14, 1, 1] ["周经理果断为了前提王先生回答问题", "something he's time right tell", "雅鹿沙", "there like that's who of were back now", "see", "a how or there back so why me", "v", "嘉运", "透露", "多点出来"] ["x", "况且", "--", "张华预计消费者正式版本身人员看重操心目前", "招待张海燕集成商朱晓明随北风含苞不敢谁知道起来", "锐成", "all", "--"] ["算错留言价格上清库存结果运费一句话十二创恒", "听着", "有波", "r", "濮阳", "-", "?", "收藏夹", "--", "一年你好吗"] [] ["for", "", "time know really can't were can all he's", "上面这一块天浩火车无关", "you're", "焦煤", "附近", "c"] ["", "视讯", "--"] ["k", "我在上呀", "水晶单子纯平直接压力早着呢副总场景"] [] ["x", "e", "r", "f", "h"] ["o", "j", "h", "p", "f", "b", "n", "r"] ["v", "f", "u", "x", "j", "p", "q", "q", "b", "n"] ["t", "c", "r", "p", "h"] [2023-12-14, 9999-12-31, 2023-12-11, 2024-02-18, 2024-01-08, 2023-12-11, 2023-12-11, 2027-01-16, 2027-01-09, 2027-01-09] [2027-01-09, 2017-09-09, 2023-12-14] [9999-12-31, 2023-12-20, 2026-01-18, 2023-12-15, 2024-01-08, 2023-12-17, 2027-01-16, 2025-02-17, 2023-12-13, 2014-08-12] [2023-12-13, 2014-08-12, 2023-12-09] [2023-12-15 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00] \N [2024-02-18 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00] [9999-12-31 23:59:59, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-11 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-17 00:00:00] [2026-01-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 2024-06-30 12:01:02.123000] [2014-08-12 00:00:00, 2023-12-20 00:00:00, 2005-04-04 00:00:00, 2013-02-21 00:00:00, 2023-12-12 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-09 00:00:00] [2023-12-15 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00] [2026-02-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00] [2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00] [2026-01-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] +161 0 440625039 32679 -1519044892 2024-07-01 2025-06-18 2027-01-16 9999-12-31 -- his no if for of could do this would can't or do my 折叠网通我那个叫价格上能用吗北环选择 -- yeah 银行退信喝得多兼容性文宁信阳广发宝龙无关 0 0 1.200000 \N 100.020000 40.123000 2024-08-03 13:08:30 2026-01-18 00:00:00 2024-01-19 00:00:00 2014-08-12 00:00:00 2024-01-09 00:00:00 2024-06-30 12:01:02.123000 2024-01-31 00:00:00 2023-12-15 00:00:00 [1, 1, 1, 1, 1] \N [1, 0, 1, 0, 0, 0, 1, 1, 0, 1] [] [-85, 1, 0] [-75, -71, 17, -83, -18, -116, -1, -103, 1, 127] [-1, -104, -128] [] [-20665, -20191, 0, -1, 12871, 32767, 1, -32768] [245, 32595, 1, -15681, 1] [-32768, -26706, 32767, -32768, -14280] [-1, -969, 1, -4642, -8808] [-1863323167, -2147483648, -1505324569] [-277154167, -688228912, 147483648, 32679, 32679] [147483648, 1437063567, 147483648, 32679, -2147483648, -1353936967, -2147483648, 219908772] [-1012547259, 147483648, 147483648, -2147483648, 466078995] [9223372036854775807, -9223372036854775808, 1757393453486256509] \N [6712307307219342533, 9223372036854775807, 0, 32679, 9223372036854775807, 1, -3560796602543532344, 32679] [9223372036854775807, -4631446, 0, 32679, 0, 9223372036854775807, -1, 2120132694, 245, 6299509882521653248] [6, 237797815, -2, 0, 1, -2, 315233838, -1] [-2, 1, 6] [] [-1458713797, -2, -1] [-1.200000000000, 60.035500000000, 0.000000000000, 92.014600000000, 80.018200000000, 100.020000000000, 300.343000000000, 40.123000000000] [17.016000000000, 40.123000000000, 40.123000000000, 0.000000000000, 64.065200000000, 22.196500000000, 300.343000000000, 14.141500000000, 54.097100000000, -1.200000000000] [0.011500000000, 0.000000000000, 95.112600000000, 41.171200000000, 2.009800000000, 1.200000000000, 37.088600000000, -1.200000000000] [1.200000000000, 53.177600000000, 0.000000000000, 300.343000000000, 40.123000000000, 38.086900000000, 100.020000000000, 15.054800000000, 0.000000000000, 300.343000000000] [-1.2000, 61.1447, 0.0000, -1.2000, 300.3430, 98.1550, -1.2000, 40.1745] [0.0000, 1.2000, 0.0000, 33.0039, 100.0200, 77.0551, 51.1156, 16.0405, 1.2000, 30.0245] [0.0000, 300.3430, 47.1189, 0.0000, 88.1786, 1.2000, 300.3430, 11.0923, 68.1443, 100.0200] [0.0000, 0.0000, 1.2000, 40.1230, 300.3430, 1.2000, 54.0398, 40.1230, -1.2000, 40.1230] [] \N [0, 300, 300, 100, 300] [300, 100, 1] \N ["this", "--", "方面又是见你", "your he's something you're from", "at see why as out"] ["--", "f", "from was yes his", "商都仔细", "?", "yeah it really", "他说你还没有回来啊", "were"] ["I'll", "?", "-", "", "t", "going", "", "原来", "and oh how okay could will on in", "?"] ["your", "i", "-", "高级", "拿出来"] ["because oh him right one go know would up I'm", "don't the back", "p"] ["with", "商贸科技大厦宠物很早多大", "花月夜来访", "?", "?"] ["锐康", "-", "?", "", "五万条做私单之间参考过来超利我到不了考勤", "志光双机热备方案制度安小不大一会吧老人家最新保密干嘛天浩", "here of there they", "i", "c", "孔艳"] ["f", "d", "d"] ["h", "v", "i", "n", "b", "e", "g", "b", "s", "t"] ["p", "z", "j", "t", "j"] ["v", "h", "w"] [2027-01-16, 2023-12-09, 2024-01-17] \N [] [] [2014-11-22 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00] [2023-12-09 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00] [2024-01-09 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00] [] [2023-12-09 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2023-01-15 08:32:59.123000] [2024-08-03 13:08:30, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00, 2023-12-20 00:00:00] [2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-07-01 00:00:00] [2024-08-03 13:08:30, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00] \N [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] [2027-01-16 00:00:00, 2023-12-14 00:00:00, 2023-12-16 00:00:00] [2025-02-17 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00] +164 145685714 2147483647 645124951 141430231 2024-08-03 2026-01-18 2023-12-12 2024-01-31 除外防火墙会议第三只宝龙完成眼睁睁 梦幻水族众人房间保持面对瑞昌伴随前几天黄黎明 硬盘 your do he's like how no time think at yes o 相信趋势天成苑管理海涛 一开始 ? \N 0 1.200000 \N 55.196300 65.040900 2024-06-30 12:01:02.123000 2025-06-18 00:00:00 2023-12-16 00:00:00 2024-01-31 00:00:00 2024-02-18 00:00:00 2024-01-09 00:00:00 2023-01-15 08:32:59.123123 2023-12-10 00:00:00 [1, 1, 0, 0, 1, 1, 0, 0] [0, 1, 0, 1, 1, 1, 0, 0, 0, 1] [0, 0, 1, 0, 0, 0, 0, 1, 0, 0] [1, 0, 0, 1, 1, 0, 1, 1] [115, -72, 1, 15, 114, -128, 2, -104, 127, -1] [] [120, -85, -128, 0, -122] [] [-7017, 3744, 32767, -19930, -1, 32767, 29138, -11033] [] [13080, -1, -12, 245, 23366] [-1, 14305, -32768, -32768, 245, 21567, 6048, 245] [-2147483648, 32679, 1, 1836683830, 32679] [-2147483648, -1377444961, 0, 1, -1494712368] [0, 0, 32679, -2147483648, 19911884] [2147483647, 1800570125, 1, -863500518, -1, 0, 2011499416, -224403138] [4171306, 32679, 147483648, 32679, 621085] [245, -2092804384, -7023643889546772405, 32679, -4377268621966149614] [] [-1907170265, -6115730, 0, 8100159, 1] [7, -2, 1, -1445334901, 0, 0, 1, 3, 9, 1] [-2, -1034193971, 7, 4, 1] [1, 1, 0, -1, -1, -177544197, -2, -2] [-2, 1, 5, -261181950, 1423950717] [43.161500000000, 57.021800000000, 99.155800000000, 0.000000000000, 0.000000000000, 36.083800000000, 300.343000000000, 1.200000000000, -1.200000000000, 0.000000000000] [100.020000000000, 49.117200000000, 17.083100000000, 300.343000000000, 33.028800000000] [0.000000000000, 33.036200000000, 40.123000000000, -1.200000000000, 100.020000000000, 96.058300000000, -1.200000000000, 36.034600000000, 63.089800000000, 98.159000000000] [1.200000000000, 40.123000000000, -1.200000000000, 1.200000000000, 1.200000000000, 40.123000000000, 100.020000000000, 0.000000000000] [40.1230, 77.0282, 300.3430, 0.0000, 94.1485, 100.0200, 300.3430, 83.1125] [1.2000, 1.2000, -1.2000, 11.1006, 0.0000, 1.2000, 100.0200, 85.1393, 40.1230, 100.0200] [36.1851, 100.0200, 100.0200, -1.2000, 1.2000, 300.3430, 98.0113, 100.0200] [0.0000, 0.0000, 0.0000, 73.1767, 43.1064, 40.1230, 28.1346, 0.0000, 40.1230, 40.1230] [56, 40, -1, 100, 0, 23, 0, -1] [0, 0, 40, 100, 61] [5, 50, 1, -1, 69, 300, 1, -1] [1, 1, 40, 100, 1, 300, 300, 4, 89, 84] ["青云", "医生", "could right not do", "say", "me time get really are i", "", "have", "汪洋若虚性格招待争取解决中关村索尼"] [] ["--", "d", "i", "不说创彩儿孙们代表办事还有你的在郑州应该", "", "", "", "尽请谅解小车前提广发有人歘俩仪器失恋"] ["封库", "right", "一辈子输入旭日", "--", "跟我发给你不认识联系新乡欣慰陪伴好梦任何", "go do had on to with I'll", "磁盘柜", "because they oh about then", "b", "--"] ["", "-", "人员名单自动那种玫瑰老师", "don't", "as", "the", "医药面对孤独减去浮动实力本就故事数据", "正式"] ["that's", "k", "车费参数表房子单位吴静前台语音别人众诚", "汉语", "-", "got didn't good who like or it he had", "", "国际企业", "感悟见到取回冰火一句话资讯保卡南风方面短语", "?"] [] ["--", "him", "次要高就左右便宜圣诞节资格则会去做起床", "something", "序列号赶紧自动那次几个", "k", "-", "from like will something", "老人家", "--"] ["h", "v", "i"] ["z", "q", "p", "s", "t"] ["m", "u", "a", "n", "i", "h", "r", "i", "w", "b"] ["j", "h", "p", "e", "r", "t", "t", "t"] [9999-12-31, 2026-02-18, 2024-01-31, 9999-12-31, 2023-12-12, 2027-01-16, 2024-02-18, 2023-12-20] [2023-12-15, 2014-08-12, 2023-12-13, 2024-01-19, 2023-12-18, 2025-06-18, 2023-12-12, 9999-12-31, 2025-06-18, 2027-01-16] [2024-01-09, 2023-12-11, 2024-01-17] [2027-01-16, 2023-12-15, 9999-12-31, 2024-06-30, 2014-08-12, 2024-02-18, 2024-01-31, 2023-12-15] [2023-12-14 00:00:00, 2016-03-13 00:00:00, 9999-12-31 00:00:00] [2023-01-15 08:32:59, 2024-01-09 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59] [2024-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00] [2023-12-14 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02, 2024-06-30 12:01:02, 2024-06-30 12:01:02, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [2026-01-18 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-17 00:00:00, 2017-11-12 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00] [2024-01-19 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123000] [] [2027-01-09 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2023-12-13 00:00:00, 9999-12-31 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00] [2023-12-15 00:00:00, 2024-01-31 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00] [2023-12-09 00:00:00, 2025-02-18 00:00:00, 9999-12-31 23:59:59] [2023-12-15 00:00:00, 2024-07-01 00:00:00, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00] +165 -424393319 1331316634 157961020 1749491463 2024-02-18 2027-01-16 2024-01-17 2014-08-12 压在 yeah I'm look 伟旭 肯自己以前数据哦三大那种农村十分县城 这种院内昌鹏这个 1 0 \N 100.020000 0.000000 1.200000 2025-02-17 00:00:00 2024-07-01 00:00:00 2023-12-20 00:00:00 2023-12-09 00:00:00 2023-12-15 00:00:00 2024-01-17 00:00:00 2024-01-31 00:00:00 2024-01-17 00:00:00 [1, 1, 0, 0, 0, 1, 1, 1, 1, 0] [0, 0, 0, 0, 0, 1, 1, 0, 1, 1] [0, 1, 1, 1, 0] [1, 0, 0, 1, 1, 0, 0, 1, 1, 1] [-1, -41, -119] [] [1, 1, -128] [-128, 49, 62, -1, -100, -128, 8, 0, -1, -1] [-28450, -20994, -32768] [-32768, 19788, 1, -4707, 1, 245, 1, 1] [32767, 24633, -31950, -17846, 0, 7894, -1, -31481] [21035, 29773, 763] [-1, 660245001, -1] [-2147483648, -1, -1, -1, -2147483648, 1, 2147483647, -1041462313, -1169295866, 0] [] [32679, 1, 133482993] [-1, -1, 6588930, -3115810145603836610, 147483648] [-5099872, 1820893175, 1807377052504611096, 1, -1] [] [] [628839929, 1, -1, -1, 1, 0, 1, -1, -2, -1] \N [2, -650816338, -1, 1, 1] [1, -1, 7, 0, -1] [48.168600000000, -1.200000000000, 100.020000000000, 0.000000000000, 300.343000000000, 18.067200000000, 0.000000000000, 40.154300000000] [0.000000000000, 93.051400000000, 100.020000000000] [-1.200000000000, 1.200000000000, 52.020300000000, 40.123000000000, 47.000400000000, 1.200000000000, 100.020000000000, 46.096200000000, 0.000000000000, 72.019700000000] [40.123000000000, 1.200000000000, 100.020000000000, 86.041500000000, 61.199300000000, 300.343000000000, 27.034300000000, 64.150500000000, 11.060600000000, 61.017800000000] [40.1230, 1.2000, 40.1230, -1.2000, 0.0000] [300.3430, 1.2000, 35.0254, 100.0200, 100.0200, 300.3430, 1.2000, 2.0902] [300.3430, 98.0201, -1.2000] [5.0785, 47.1143, 57.1741, 25.0972, 58.0793, 26.1255, -1.2000, 1.2000] [300, -1, 86, 76, 0, 67, 100, 300] [94, 68, 89] [-1, 1, 40] [33, 79, 3, 0, -1, 40, 27, 300] ["or", "he's", "以为你一会简历不能启用名家", "back", "c"] ["do as know a one want then will", "--", "本就", "-", "捷创煤气改动懂得高级总代理羡慕事业"] ["-", "?", "忘记不了"] ["-", "be", "-", "--", "几篇", "原则来看", "苏小明了解还行妹妹道理汉字杀毒邮政附件暴风影音", "about"] ["got from his some this are of because when think", "if you okay time", "the", "尽量", "for it's I'll can't would say there that can be", "-", "小徐", "下班", "can't", "彭伟"] ["-", "-", "海豚"] [] ["众诚", "中国三件姐姐航海路你家随便浪漫牌子刘汉", "世上", "for", "王星"] ["y", "l", "p"] ["o", "p", "y", "p", "o", "g", "m", "r", "l", "m"] ["l", "h", "v", "n", "n", "b", "w", "k"] [] [2025-02-17, 2024-01-19, 2024-02-18] [9999-12-31, 2023-12-11, 2024-01-17] [] [2024-02-18, 2023-12-20, 2023-12-13] [2023-12-11 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-10 00:00:00] [2023-12-17 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 2026-01-18 00:00:00, 2010-02-21 00:00:00] [2026-01-18 00:00:00, 2023-01-15 08:32:59, 2023-12-17 00:00:00] [2024-01-08 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00] [2007-02-12 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00] [] [2026-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00] [2024-08-03 13:08:30, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2024-08-03 13:08:30, 2013-09-19 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-12 00:00:00] [] [2026-02-18 00:00:00, 2023-12-11 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2024-01-31 00:00:00] +166 -1 \N -1 86364983 2023-12-17 2023-01-15 2026-01-18 2024-02-18 going were how well could that's time to don't 检测长信晚上给把疑问一次性创新高性能 k 如果兼得我在郑州份款化验 ? we \N 0 77.102800 58.167300 -1.200000 62.085700 2024-01-19 00:00:00 2026-02-18 00:00:00 2023-12-09 00:00:00 2024-01-08 00:00:00 2024-07-01 00:00:00 2023-12-17 00:00:00 2027-01-09 00:00:00 2027-01-09 00:00:00 [0, 0, 0, 1, 0] [0, 0, 0] [1, 0, 0, 0, 0] [1, 1, 0, 1, 1, 0, 0, 0, 0, 1] \N [-128, -24, 0] [-91, -128, 9, -128, 1] [38, 100, 0] [-32768, 30871, -14397, -26576, -32768, -1, -22505, 1, 1, 30679] [26739, 10265, 29030, 32270, -28163, 245, 2977, -20102] [-32768, 24599, 32767, -32164, -12404, -6881, 6807, 14476] [245, -4265, 32767, 32767, -26493, 12817, 26631, 19386] [-2126080334, -1, 1] [-1, 1525085976, 507334874, -1, 1835543798] [373376558, -2147483648, 147483648, 147483648, 32679, 147483648, 147483648, 32679, -2147483648, 2147483647] [-2147483648, 0, 845367398] [32679, 9223372036854775807, -918379692181538629, 1, 32679] [] [-263923699, -3703071270388550710, -4828913610274964454] [-1, -5466447122773049391, -1, 679726674600489903, 1] [1, 1, 1522723777] [-2, 1, 1, 1, -2] [5, -1, -1, 1, 0, 0, 2134027323, -1] [-2074212916, 7, -1, -2, -1, 1, -1373663340, 1] [-1.200000000000, 0.000000000000, 18.110500000000, 0.000000000000, 300.343000000000, 300.343000000000, 300.343000000000, 300.343000000000] [0.000000000000, 15.195500000000, 100.020000000000] [100.020000000000, 100.020000000000, 93.033200000000] [300.343000000000, 11.067200000000, 34.192000000000] [100.0200, 1.2000, 65.1818, 39.0788, 7.1769, 0.0000, 300.3430, -1.2000, 52.0048, 1.2000] [76.1429, -1.2000, -1.2000, 40.1230, 300.3430, -1.2000, 28.1889, 40.1230, 0.0000, 100.0200] [100.0200, 0.0000, 300.3430, 99.1590, 40.1230, 89.1753, 98.0250, 30.0207, 1.2000, -1.2000] [300.3430, 300.3430, 71.0111, 43.1042, 300.3430, 1.2000, 300.3430, -1.2000] [0, 1, 91, -1, -1] [300, 300, 3, 100, 99] [-1, 1, 0, 50, 100, 62, 58, 2, 66, 17] [79, 1, 0] ["me", "so", "深入私人香港同方", "out me from", "-"] ["-", "工作站虚妄你能看到应该做过", "不了申腾卖友镜面张经理女孩衣服规模", "跑跑卡丁车", "--", "--", "?", "-"] [] ["--", "人情反对新的未定暴力小王", "when get good good there up with that", "did", "-", "can when got time well from no ok", "寻找拦不住下着", "got", "mean out tell to have back for me here", "well mean me her had i know to not can"] ["", "are when one we then be mean be tell mean", "j", "是你呀小丽伤和气主要奖励", "东方因为", "go but how why all with", "你的", "?"] ["l", "say", "公道话", "难说", "奖金不愿意颜色赞美诗老人家"] ["?", "it", "c"] [] [] ["d", "n", "y"] ["l", "l", "u", "l", "x", "y", "q", "j", "c", "q"] ["q", "t", "q", "k", "g", "l", "e", "v"] [9999-12-31, 2023-12-10, 2023-12-17] [2023-12-13, 2023-12-11, 2024-01-08, 2024-06-30, 2024-08-03, 2023-12-10, 2024-01-08, 2024-06-30, 2007-06-16, 2027-01-16] [2023-12-12, 2024-01-31, 2023-12-17, 2023-12-10, 2025-02-17, 2027-01-16, 2023-12-15, 2023-12-15] [] [2026-01-18 00:00:00, 2008-07-01 00:00:00, 2024-01-17 00:00:00] [2024-06-30 12:01:02, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2023-12-13 00:00:00] [2026-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00] [2024-01-09 00:00:00, 9999-12-31 23:59:59, 2023-12-19 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00, 2024-06-30 12:01:02, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00] [2027-01-09 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00] [2025-02-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2026-02-18 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00] [9999-12-31 23:59:59, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00] [2014-08-12 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00] [9999-12-31 23:59:59, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00] [] [2024-02-18 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123123, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-01-19 00:00:00] +167 32679 1497298360 -2147483648 -1 2025-06-18 2025-06-18 2023-12-20 2024-01-19 圣荣 退信 were some as is it have 含义 this from 0 0 0.000000 -1.200000 40.123000 63.112700 2023-12-11 00:00:00 2026-02-18 00:00:00 2024-02-18 00:00:00 2014-08-12 00:00:00 2027-01-16 00:00:00 2026-02-18 00:00:00 2023-12-14 00:00:00 2027-01-09 00:00:00 [0, 1, 1, 0, 0, 1, 1, 1] [1, 0, 1, 1, 1, 1, 0, 0] [0, 1, 0] [0, 0, 0] [1, 1, 8, 1, 2, 23, -33, 0] [] [-99, -128, 127, 6, 1, 105, -128, 127] [-70, -1, 28, 30, -120, -64, 1, -60] \N [-1, 18457, 1, 29047, 32767, -27070, -1, 245] [] [32767, 18515, 31960, 31678, -7901] [-247221265, 1806038199, -395402707] [-1980188065, -2147483648, -42035516] [-77012516, -1421629030, -1] [] [245, 147483648, 245, 32679, -9223372036854775808, 32679, -1721173849509804772, -1664511848] [32679, 9051648602572835095, 1238198866, -6641651, -9223372036854775808, 245, 0, -863053670, 454303091, -1156380949] [3211805, 147483648, 245] [9223372036854775807, 4984765499868749796, 7534252, 32679, 1, -9223372036854775808, 1148150698617804770, 32679, 9223372036854775807, 9223372036854775807] [1, -2, -2, 8, 1] [1, -1, 0] [1, -1, 1, 4, 6, 4, -2, -2] [-2, -1, 3] [100.020000000000, 91.040800000000, 40.123000000000, 1.200000000000, 0.000000000000, 300.343000000000, 300.343000000000, 300.343000000000, 40.123000000000, 8.046900000000] [40.123000000000, 1.200000000000, 0.000000000000] [] [-1.200000000000, 100.020000000000, 0.000000000000] [-1.2000, 300.3430, -1.2000, 54.1836, 94.1146, 0.0535, 16.0452, 94.1092] [12.0200, 40.1230, 0.0000, 300.3430, 1.2000, 96.0276, 91.0696, 72.1534, 56.0141, 40.1230] [100.0200, 1.2000, 300.3430, 0.0000, 300.3430, 40.1230, 98.1863, 100.0200, 48.0243, 40.1230] [-1.2000, 80.1248, 33.1799] [-1, 100, 95, 0, 300, 1, 1, -1] [] [-1, 1, 95, 90, 0] [300, 62, 0] ["什么时间磁带面前提呢", "百脑汇胡华威就是反对", "--", "丁丁", "?", "do", "洛宁扩充华栋低端有限失误", "", "-", "time here"] [] ["--", "m", "交通影子招待费吃住同事", "--", "", "I'll", "光驱", "郑州市", "打过", "?"] ["you're got and", "?", "it are and as but i", "简介兄弟徐敬东金牌经销总代理请客周经理速度消息金色", "k", "意见每一小时忘记百家", "?", "下一部分追到世纪城理解南三环玩玩想要主做这场价格上", "?", "驻马店"] ["we so I'm", "", "it", "", "没钱", "m", "许昌合适这么图片系诶邪恶打字门票岩春见见", "me do"] ["okay do good there we that", "", "good", "-", "", "?", "--", "?", "me", "-"] ["欧阳态度大风地方黎明达成率", "a get we oh get hey know", "can", "不方便", "q"] ["look", "oh", "say had oh something no have can't"] ["g", "a", "n"] ["d", "m", "e", "q", "b"] ["r", "o", "c", "r", "l"] ["q", "z", "s", "m", "z", "k", "g", "k", "c", "t"] \N [2024-01-08, 2024-01-09, 2024-02-18, 2024-01-31, 2026-01-18] [] [9999-12-31, 2023-12-15, 2024-01-31] [9999-12-31 23:59:59, 2014-08-25 00:00:00, 2023-12-19 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00, 9999-12-31 23:59:59, 2023-12-12 00:00:00] [2026-02-18 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-01-08 00:00:00, 2024-01-08 00:00:00] [] [2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00] [] [2023-12-16 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-31 00:00:00, 2007-11-25 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2024-01-09 00:00:00, 2023-12-17 00:00:00] [] [9999-12-31 23:59:59, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2014-08-12 00:00:00] [2023-12-09 00:00:00, 2024-01-19 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00] [2024-07-01 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00] [2019-01-23 00:00:00, 2023-01-15 08:32:59.123123, 9999-12-31 00:00:00] +168 2147483647 147483648 -540728613 -171229567 9999-12-31 2023-12-13 2023-12-19 2023-12-15 -- 大雪新拓电池集成商金鼎难说结果天翔 一条提供 可选 and 不多 报价有过内外一般 1 1 300.343000 100.020000 300.343000 0.000000 2026-01-18 00:00:00 2025-02-17 00:00:00 2024-08-03 13:08:30 9999-12-31 00:00:00 2026-02-18 00:00:00 2019-01-25 00:00:00 9999-12-31 00:00:00 9999-12-31 23:59:59 [1, 0, 1, 0, 0, 0, 0, 0] [0, 1, 1, 1, 0, 0, 1, 0, 0, 1] [1, 1, 1] [0, 1, 0, 1, 0] [] \N [] [1, 1, -128, 3, -118] [9743, -27878, 7700, -32768, 1, 0, 0, -1, -1, -1] [16237, 245, -18159, 16863, 1, -1, -22418, 4387] [1, -19509, 21561] [0, 32767, 31052, 245, 32767, -5977, 0, 245, -22255, 1580] [-46454358, -1, 1, -1964855846, 2147483647, 2147483647, 590254820, -337188457, 0, -609688857] [1209082371, 0, -379075641, -117212886, 0, -412253976, 147483648, -2117641461] [] [255828335, 147483648, 1314002975, -1, -1187258911, 2147483647, 227605674, 136615830, 1, 1342910195] [9223372036854775807, 32679, 245] \N [9223372036854775807, 9223372036854775807, -9223372036854775808, -9223372036854775808, 245] [] [-1940786004, -2, 1, -709947324, 0, 2, 1, -1146934870] [0, 4, 1, 6, 7, -2, -2, 0] [-1, 1, -794168804] [-2, -2, -1, 8, -1, -981574832, 1, 1, -2, 4] [300.343000000000, 40.123000000000, 97.022800000000, 95.187900000000, 300.343000000000, 40.123000000000, 1.200000000000, 100.020000000000, 8.126000000000, 300.343000000000] [40.123000000000, 300.343000000000, -1.200000000000] [0.000000000000, 0.000000000000, 23.056200000000] [81.029800000000, 100.020000000000, 100.020000000000, 9.140600000000, 98.114600000000, 40.123000000000, -1.200000000000, 70.127900000000] [59.1274, 1.2000, 300.3430, 100.0200, 1.2000] \N [40.1230, 3.1102, 63.0109, 43.1140, 300.3430, 1.2000, 44.1878, 11.1044] [] [100, 100, 40, 68, 100] [] [64, 56, 75, 40, -1, 37, 300, -1] [1, 100, 100, -1, 40] ["-", "多家", "were in who mean we he can't there some with", "v", "有谁", "佳成放假皮肤客户一台看课件素材做生意下图东西", "加密狗", "易插拔", "?", "海民垃圾帮忙否则走在"] ["其人", "up", "could"] ["yeah okay been had", "?", "来访轻信知道了机柜绿城大山那六", "--", "-", "?", "靠前使用只要发布", "-", "your as you something mean his I'm", "o"] ["that's", "条数", "", "?", "容易游民期间钻石金牌通用作风请问"] ["--", "?", "西郊"] [] ["头痛一段系诶邪恶高一些看会等等剑派无法回来新宇", "if know you're with there up this now", "", "okay", "m", "time could me think one what her from", "all", "okay", "?", "--"] [] ["k", "g", "m", "v", "d", "f", "t", "i"] ["s", "k", "e", "f", "t"] ["z", "n", "w", "t", "i", "y", "u", "o"] ["t", "l", "p"] \N [2024-01-19, 2024-01-19, 2026-01-18, 2024-01-17, 2024-02-18, 2026-02-18, 2023-12-17, 2025-06-18, 2027-01-16, 2023-12-10] [2023-12-11, 2023-12-13, 2023-12-11, 2023-12-20, 2024-01-17] [] \N [2024-07-01 00:00:00, 2024-08-03 13:08:30, 2024-07-01 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2024-07-01 00:00:00] [2024-08-03 13:08:30, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2023-01-15 08:32:59, 2023-01-15 08:32:59] [2023-12-15 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59] [] [2025-06-18 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-10 00:00:00] [2023-12-15 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00] [] [2023-12-12 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2025-06-18 00:00:00] [9999-12-31 00:00:00, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00] +169 147483648 1528347212 147483648 -2147483648 2025-02-17 2025-02-17 2024-01-09 2023-12-10 我找同学不要错思科王东亮划分认为卡巴斯基 ? -- did 正式版前几天孙娟有点参观火车站 - ? ? \N 0 21.080200 45.143000 37.124900 91.064900 2023-12-10 00:00:00 2023-12-17 00:00:00 2023-12-18 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2023-12-15 00:00:00 2023-12-20 00:00:00 2023-12-16 00:00:00 [1, 1, 1, 0, 1, 0, 0, 0] [0, 0, 0, 1, 0, 1, 1, 1, 1, 1] [1, 0, 0, 0, 1, 1, 1, 0] [0, 0, 0, 0, 0, 0, 1, 0, 0, 0] [-128, 0, -1, 39, -1, 1, 1, -128] [] [121, 1, 0, -1, 0] [1, 30, 0] [-31511, -2193, 245, 1, -1] [-1, 0, 1] [-31166, 5196, -21258] [0, 32767, -2743, -1, 32767, 0, -1, 32767] \N [] [-1729814003, 1474136522, -2147483648, 32679, -492495447, -1256363520, -2147483648, -1, -1, 644793540] [2147483647, -1045501765, 32679, 32679, -848917947, 147483648, -1, 32679] [-5634811568144991326, 2300648216080778856, 1, 0, 1, 9223372036854775807, 9223372036854775807, -2257940707609422099, -9223372036854775808, -9223372036854775808] [5524995016182671037, 209589749877108533, 0] [6161485, 1, -126874889, -2222473, -185970931, -2250412, 609376699314553890, 7856482370834141386, -9223372036854775808, -1] [3211533847271521687, 9223372036854775807, 147483648] [] [-391443465, 1, 1] [-2, -2, -1126817377, 1, -1607224707, -1405796817, 6, 4] [] \N \N [40.123000000000, 100.020000000000, 1.200000000000, -1.200000000000, 40.123000000000, 40.123000000000, 40.123000000000, 58.115200000000] [-1.200000000000, 12.182200000000, 100.020000000000] [100.0200, 40.1230, 87.1697, 29.1708, 40.1230, 28.0647, 40.1230, 0.0000] [] [40.1230, 100.0200, 5.1312, 33.0636, 42.1830, 300.3430, 1.2000, 300.3430] [40.1230, 1.2000, 300.3430] [] [60, 100, 100, 100, -1] [8, -1, 50] [13, -1, 100] ["捷创时间少舞阳泰安差价", "of", "国栋跃民不在", "又出帮助王总签不签都早上安排一周摄像头谢谢", "--"] ["-", "a", "there", "", "?", "e", "say", "can't"] [] [] ["四方有你们厉害允许水哦去年其他名字", "", "of think something and", "你是九洲降低你好预防联恒", "right", "d", "we", "往事也在展开大会泡茶写上去一次", "were look up you're go", "透明注定机柜顺风呀库存"] [] ["--", "the really", "u", "核对过", "?"] ["瑞达", "yes", "-"] ["t", "q", "e"] ["q", "o", "u", "w", "k", "i", "x", "f", "x", "t"] ["j", "z", "d", "a", "e"] ["v", "n", "y", "k", "z"] [2024-01-31, 2023-01-15, 2024-02-18, 2014-08-12, 2024-02-18, 2012-06-19, 9999-12-31, 2023-12-11] [2023-12-15, 2024-01-19, 2024-01-17, 2023-12-10, 2023-12-16] [2024-01-17, 2026-01-18, 2027-01-16, 2023-01-15, 2025-02-17] [2024-01-09, 2023-12-16, 9999-12-31, 2023-12-14, 2024-07-01, 2024-01-31, 2027-01-16, 2024-06-30] [9999-12-31 23:59:59, 2024-07-01 00:00:00, 2024-06-30 12:01:02] [2024-01-31 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2024-06-30 12:01:02] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30] [] [2024-07-01 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00] [2014-08-12 00:00:00, 2026-01-18 00:00:00, 2023-12-17 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00, 2018-10-15 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00] [2023-12-11 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-17 00:00:00] [2026-01-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123123, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2024-01-17 00:00:00] [9999-12-31 00:00:00, 2023-01-15 08:32:59.123123, 2004-02-21 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00] [2025-02-17 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00] [2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2014-08-12 00:00:00] +170 1010480646 -1645647590 957690185 147483648 2023-12-15 2023-12-16 2023-12-19 2023-12-11 热卖中 go because really from something b ? ? ? she want not been his now yeah could the to 1 1 100.020000 63.065900 1.200000 42.088500 2023-12-17 00:00:00 2024-06-30 12:01:02.123000 2023-12-12 00:00:00 9999-12-31 00:00:00 \N 2023-12-19 00:00:00 2014-08-12 00:00:00 2024-02-18 00:00:00 [0, 0, 1, 0, 1, 1, 0, 1] [0, 1, 1, 1, 1, 0, 1, 1, 1, 0] [1, 0, 1, 0, 1, 0, 0, 0, 0, 0] [1, 0, 1, 0, 0] [127, -1, 0] [0, 66, -128] [8, -102, -2, -128, 6, 1, 7, 127, -62, -102] [59, -128, 1, -128, -128, 127, -1, -87, 0, -128] [245, -32768, -32768, 1, -1] [32767, -1, -20090, 245, 1, -22599, -26865, 1] [0, -29295, -31067, 11028, 1428, -32768, 1, 245] [] [1244513636, 421087895, -2076955855, 759322999, -1552654962] [1, -1, -1168430192, -361610273, 1115366481, 147483648, -1, -1] [-2147483648, 32679, 0, 998200744, 585219228, 147483648, 147483648, 147483648] [592977464, -809308470, 147483648] [] [0, 245, 147483648, -6133071696240633006, -1469498184037481525, 9223372036854775807, 32679, 32679, -5388265876000760746, -4187790] [0, -8870052642026844426, -8928325903736619156, 1762337581, 9223372036854775807] [-5462113204325438747, 5250664626430491973, -7386708172716932232, -7935052, -6414237731578239113, 0, 2438760859416277888, 245] \N [9, 2, 1846370100, -1, -1, 1, -2, 0] [-1, -2, 369039689, -2, -2, -1, 0, -2] [1, -2, 1, 1, -1, -1, 1, 1, -2, 1524018234] [2.083600000000, 40.123000000000, 1.200000000000, 300.343000000000, 100.020000000000, 92.056300000000, 1.200000000000, 100.020000000000, 300.343000000000, 32.015000000000] [-1.200000000000, 1.200000000000, 0.000000000000, 1.200000000000, 17.115900000000, 1.200000000000, 300.343000000000, 300.343000000000, 81.130700000000, 45.046500000000] [1.200000000000, 0.000000000000, 0.000000000000, 23.003000000000, -1.200000000000, 1.200000000000, 40.123000000000, 0.000000000000] [] \N \N [-1.2000, 95.1614, 76.1315, 84.0624, 0.0000, 40.1230, 40.1230, 73.0309, 40.1230, 300.3430] [100.0200, 81.0843, 1.2000, 67.1107, 48.0204] [10, 100, 300, -1, 66, 94, 5, -1, -1, -1] [1, 100, 13, 83, 84, 52, 0, 40, 98, -1] [33, 15, -1, 300, 100, 64, 40, 1] [40, 1, 1, 71, 40, 1, 11, 42, 68, 22] ["r", "--", "up see can't were all will just as", "-", "-", "just mean oh", "r", "先达有些时候准确请你吃饭坐在完美考前程序", "妈的", "want"] \N ["okay", "and", "he's really i like now don't well on", "工分儿", "r", "time but what about it's do", "why", "?"] ["", "are not I'll have", "would", "j", "", "-", "all all yes he right were to had", "then", "like come who mean", ""] ["going", "up", "going so a then get", "华北区我在上呀注意", "高价跟进怎么就在在你是最近忙什么呢", "--", "-", "o", "from", "will got a"] ["now one or we", "come one tell", "开发小庞起来道理问世写上去列表沟通聚会期限", "退信", "hey", "名家", "独显", "?", "as he's a now mean no we who did", "m"] ["-", "-", "-", "计算机", "n", "?", "这次", "are"] ["长光不做儿时张总索尼河南总经销塔式灵生帮帮三石", "我借你", "通信宝宝双机热备方案不方面报价", "本就各个说好信海不欢迎听着规模网通播放", "think", "now", "张姐", "no didn't would in come well but something", "her tell I'll she there didn't are mean because her", ""] ["x", "q", "x", "m", "i"] ["o", "g", "u", "p", "e"] ["o", "b", "g", "f", "s", "j", "t", "k", "g", "w"] ["y", "q", "b", "g", "l", "z", "z", "s"] [2023-12-11, 2024-02-18, 2023-12-19, 2024-02-18, 2025-06-18] [] [2023-12-14, 2023-12-15, 2019-08-18, 2024-02-18, 2023-12-10, 2027-01-09, 2024-02-18, 2024-06-30, 2014-08-12, 2027-01-16] [2026-02-18, 2025-06-18, 2023-12-09, 2027-01-16, 2023-01-15, 2024-06-30, 2026-02-18, 2024-06-30, 2027-01-09, 2023-12-17] [9999-12-31 23:59:59, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2023-12-11 00:00:00] [] [2025-06-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00] [2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-13 00:00:00, 2023-01-15 08:32:59, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] [2024-07-01 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2008-02-28 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00] [2026-01-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2026-02-18 00:00:00, 2024-01-08 00:00:00, 2024-01-09 00:00:00, 2006-01-01 00:00:00] [2023-12-13 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2001-03-16 00:00:00, 2023-12-10 00:00:00, 2023-12-16 00:00:00] [] [2016-10-06 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00] [2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00] [2027-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000] +171 -2147483648 0 -549587565 -1890968045 2023-12-09 2026-01-18 2024-02-18 2024-01-31 - for if your i I'll when me when there got -- I'm the if do as a in 回访推荐月内木马治疗那层次招待宝源证明 she mean in my will were you're been it 0 0 70.155700 100.020000 1.200000 16.035400 2024-07-01 00:00:00 2023-12-11 00:00:00 2023-12-17 00:00:00 9999-12-31 23:59:59 2023-12-13 00:00:00 9999-12-31 23:59:59 2024-01-19 00:00:00 2025-06-18 00:00:00 \N [1, 0, 1] [0, 0, 1, 1, 1, 0, 1, 0, 0, 1] [1, 0, 0, 0, 0] [0, -1, 6, -128, 127] [-128, 63, 1, -16, -56] [0, 0, 1, 72, -1] [1, 9, 1, -83, 7, -128, -1, 127, -33, 1] [-1, -27287, -6332] [1, -32768, 3666] [] [-32768, 32767, 1] [-2147483648, 1236280375, 1565857855, 147483648, -1169396814, 1748619866, 2147483647, 685166936] [1367069103, 32679, 0, -2147483648, 1177401874] [2147483647, -2147483648, -2147483648] [-1, 147483648, 870729954, 2147483647, -2147483648] [1, 32679, -1079757795, 245, 245, -1924056144, -4019260167778487908, 32679, 147483648, -1] [7167740, -325051870, 9223372036854775807] [9223372036854775807, -3951219030901692307, -9223372036854775808, 9223372036854775807, -2924745649334090970, -1991450493, -9223372036854775808, 1165880660443576105] [-1668373720, -1274178542, 32679, 2393293, 7442126620176103199, 32679, -5343168557330781287, 0] [0, 79490857, -1, 0, -1266572070, 1, -1956957454, 1, 0, -2] [1083003271, -2, 5, 1293213934, -2] [2, 1, 1, 1, 7, 0, -1, -951351038] [1, -1, 0, 0, 1324953573] [300.343000000000, 79.178000000000, 10.139600000000] [100.020000000000, 40.123000000000, 29.007100000000, 100.020000000000, 1.168200000000] [] [40.123000000000, 1.200000000000, 1.200000000000, 98.058600000000, 300.343000000000, 0.000000000000, 1.200000000000, 78.174300000000] [91.1761, 300.3430, -1.2000, 38.0115, 300.3430, -1.2000, 1.2000, 12.0183, 1.2000, 91.1300] [300.3430, 60.1932, 100.0200, 23.1308, 300.3430, 100.0200, 100.0200, 41.1531, 1.2000, 56.0626] [40.1230, 66.0912, 65.0024] [] [-1, -1, 100, 100, 52, 19, 1, 300] [3, 42, -1, 1, 300, 64, 1, 40] [300, 0, 300] [0, 50, 100, 20, 40, 84, 300, 100] ["-", "个哦", "-"] ["那你", "聚兴", "?", "on she really", "娃娃擅作主张楼下认为等于赵经理幸福移动", "大法师告诉大家官方好句男人欢迎", "hey", "这中间好人坏人明天会更好"] ["?", "hey really want was did don't say out think", "走过", "?", "come", "c", "?", "天天山东焦作市"] [] ["--", "弹出", "if she about yeah you're he's it's"] ["攻防看是礼物刘海共享肥肉", "房间散落的", "now", "", "--"] ["be were but good see got will oh", "from", "", "mean", "", "客运", "去爱", "-"] ["could up how mean was it's we", "亿欧元", "?"] ["z", "n", "r", "f", "k", "r", "g", "d"] ["k", "f", "k", "f", "d"] ["e", "j", "y", "z", "m", "n", "w", "e"] ["m", "l", "j", "j", "m"] [2023-01-15, 2023-12-12, 2023-12-10, 2027-01-16, 2023-12-20] [2023-12-09, 2023-12-11, 2023-12-19] [] [] [2027-01-16 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00, 2026-02-18 00:00:00, 2024-06-30 12:01:02, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00] [2025-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00] [2023-12-09 00:00:00, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-16 00:00:00, 2024-01-17 00:00:00] [2023-12-14 00:00:00, 2024-06-30 12:01:02, 2017-07-16 00:00:00] [2024-01-31 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2024-07-01 00:00:00, 2024-08-03 13:08:30, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00] [2024-06-30 12:01:02.123000, 2024-07-01 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00] [2025-02-17 00:00:00, 2024-01-31 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-18 00:00:00, 2003-06-16 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00] [2024-01-17 00:00:00, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-19 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-16 00:00:00, 2023-01-15 08:32:59.123000] [2023-12-11 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00] \N [9999-12-31 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-16 00:00:00, 2024-07-01 00:00:00] [2027-01-09 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-01-17 00:00:00, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00] +172 -443630090 1622176237 147483648 -1748775009 2023-12-15 2023-12-20 2023-12-19 2024-08-03 f see 燕龙元旦注定 难道 now one because ? see didn't are as didn't didn't 时机 \N 0 67.190000 1.200000 43.126000 93.039800 2023-12-09 00:00:00 2023-12-14 00:00:00 2024-06-30 12:01:02.123000 2027-01-16 00:00:00 2027-01-16 00:00:00 2023-12-19 00:00:00 2025-02-18 00:00:00 2023-12-14 00:00:00 [0, 0, 0, 1, 0, 1, 0, 1] [1, 1, 1] [0, 0, 0, 1, 1] [1, 0, 1, 1, 0, 1, 0, 0, 1, 1] [82, -128, -1] [-1, 0, 114, 1, -85] [] [127, 55, 1, 0, 127] [0, -1, 27891, 11327, 32767, 32767, 245, -19153] [245, -32768, 0, 32636, 245, 14438, 21366, -18123] [] [-1, 12709, 31959, 245, -1, 1, 24834, 32767, 245, 32767] [147483648, 1426899957, -835951404, 1911467231, -1628282376, 0, 0, 1378142344, 1460936191, 1] [1615651728, 1118081578, 32679, -409318336, -1688352539, 1156705316, 0, 2147483647] [-1, -865113358, 0, 32679, 160084595, -926018697, 2085252142, 2147483647] [-1, -2147483648, 1, 0, -2013906759] [] [9223372036854775807, -9223372036854775808, -1566404375, 1433589607285101448, -685335657834750950] [8793942409370944679, 9223372036854775807, 0, 8142925148375320416, -312392222340273858, 32679, 147483648, 8561371880432438180] [-8394461748552374844, 147483648, -1] \N [-2, -1911705559, 0] [1, 1, -2, -1, -2] [0, -2, 0, -778453574, 9] [] [] [1.200000000000, 0.189600000000, -1.200000000000] [96.020300000000, 100.020000000000, 81.059300000000, 40.123000000000, -1.200000000000] [] [1.2000, 5.1995, 66.1126, 76.0367, -1.2000, 61.1948, 100.0200, 28.0091, 300.3430, 9.1162] [17.0546, 300.3430, -1.2000, 68.1832, 300.3430, 70.1469, 40.1230, 100.0386] [76.0722, -1.2000, 40.1729, 62.0475, 40.1230] [] [0, 40, 300, 1, 40, 100, 300, -1, 22, 16] [0, 67, 300, 0, 86, 100, 40, 40, 80, 45] [100, 0, 40] ["--", "k", "", "--", "", "其中欧", "it why go right or like it's some look this", "d"] ["明基小机器见到企业尤其性价比", "?", "so for", "?", "总价电话本邮件私聊"] [] ["有空吗凡是晚上聊南村", "look just now on are", "三润电话给我快点身份证许昌特性", "q", "think"] ["?", "里面几天就走不方便擅作主张", "-", "?", "手册"] ["一帆意义", "it", "r", "m", "for"] ["ok", "me go right was why the here like don't with", "有项目", "", "水货开出来利落好多制作我不知道你是谁开头东大街出团预算", "right", "that's", "is want him like all you because do come", "mean", "当作"] ["?", "o", "c", "", "of", "b", "正好", "get think were your oh for look get oh for", "get a me ok", "w"] ["n", "n", "v"] ["y", "b", "s"] ["d", "v", "z", "q", "e"] [] [2024-07-01, 2023-12-13, 2027-01-16] [] [2023-12-15, 2024-08-03, 2024-02-18, 2023-12-14, 2023-12-14, 2024-01-09, 2023-12-17, 2027-01-16, 2027-01-09, 2023-12-17] [] [2025-02-18 00:00:00, 9999-12-31 00:00:00, 2024-06-30 12:01:02, 9999-12-31 00:00:00, 2014-08-12 00:00:00] [] [2015-04-05 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [9999-12-31 23:59:59, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2016-06-15 00:00:00, 2023-12-15 00:00:00, 2024-01-17 00:00:00] [2024-06-30 12:01:02.123000, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-17 00:00:00, 2023-12-09 00:00:00, 2023-12-14 00:00:00] \N [2023-12-15 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-10 00:00:00, 2001-03-17 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123000] [] [2025-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30] \N [2024-01-08 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2023-12-12 00:00:00, 2024-01-31 00:00:00] [] +173 -1074115956 1948241888 32679 147483648 2024-06-30 \N 2024-02-18 2023-12-16 o 元旦小雪需要 know 有项目 名杂焦煤 t 贸易没人心里过的好吗东海抓住 \N 1 66.067800 300.343000 92.040400 8.087100 2024-07-01 00:00:00 2023-12-19 00:00:00 2023-12-12 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2023-12-12 00:00:00 2027-01-09 00:00:00 2023-12-11 00:00:00 [] [0, 0, 0, 1, 0, 0, 0, 0, 1, 0] [1, 1, 0] [0, 1, 1] [-128, 67, 0, 127, -48, 91, 2, -128] [-104, -128, 6] [105, -128, 84] [8, -128, 44, 0, -1, 7, -1, 116] [10383, -27794, 4093, 1265, 1, 22613, 245, -8999, 32767, 31819] [-32768, 1, -1] [-27839, 2467, -21745, -32768, 1, 32767, -12689, -1, 2839, 29703] [] [0, -1, -2147483648, 613262993, 1433903383] \N [0, -2147483648, -1] [-2074770605, -2147483648, 32679, 1369687622, 1771763815, 32679, 32258695, 388364574, 2147483647, 653570188] [9223372036854775807, 147483648, 4466094661407214381, 8922108646870214962, -1, -1435751362418312886, 32679, 9223372036854775807] [32679, 9223372036854775807, 0, 32679, 147483648, 147483648, -1254315579, 117279692] [] [424481653, -1, 32679, 346417958647131404, 1, -509525588062141944, 499463599389260776, 0] \N [1, 0, -1, 1, -1058413857, -1, 8, -1, 1, -1] [] [1, 0, 1] [40.123000000000, 70.178600000000, 6.177900000000, 33.027000000000, 0.000000000000, 1.200000000000, 86.170000000000, 300.343000000000, 100.020000000000, 71.079400000000] [13.075400000000, 0.000000000000, 40.123000000000, 41.063800000000, 40.123000000000] [] [48.137900000000, 40.123000000000, 3.083900000000, 31.098600000000, 1.200000000000, 300.343000000000, 0.000000000000, 1.200000000000] [0.0000, -1.2000, 100.0200, -1.2000, 44.0280, 0.0000, 5.1664, 40.1230] [63.0515, 0.0000, 7.1350] [60.1561, 40.1230, 6.0311, 40.1230, 300.3430, 0.0000, -1.2000, 300.3430, 300.3430, 85.0388] [-1.2000, 1.2000, 65.0760, 1.2000, 40.1230, 1.2000, 1.2000, 100.0200] [300, 42, 24, 32, 90, 40, 8, 14, 8, 90] [1, 29, 40] [4, 100, 0, 300, 300, 3, 99, 100, 300, 63] [40, -1, 69] ["", "活动资讯忘不了令我制造有关老同学", "属于", "", "was but for when his"] ["d", "李经理不想看着科嘉交货硬件任务工资区域我打印", "那六", "-", ""] ["?", "在他家不来柒牌终究", "-", "--", "白象肥肉贵姓一千多安富还好吧重新发嘉运达青少年", "--", "尾货", "to"] ["上网", "天数", "--"] ["宇瑞一台招待费招生位居谁在", "--", "that's", "is", "?"] ["-", "--", "不问代理价格志彬图形型号大忙人黑色身份证绿城的风格"] [] ["for", "这辈子光缆实现处于资料羡慕既然看到绿洲离谱", "you're we I'll did were well can", "-", "or at if tell now for like"] ["o", "q", "u"] ["q", "b", "m", "o", "s", "o", "d", "m"] ["e", "s", "m", "t", "x", "z", "u", "g"] ["l", "r", "k", "f", "r", "d", "v", "d"] [2023-01-15, 2024-02-18, 2023-12-15, 2025-06-18, 2023-12-09, 2024-02-18, 2024-01-31, 2024-01-19] \N [2025-02-17, 2024-02-18, 2027-01-16, 2026-02-18, 2023-12-09, 9999-12-31, 2023-12-16, 2026-02-18, 2024-01-09, 2025-06-18] [] [2026-01-18 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00] [2023-12-11 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02] [2025-02-17 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-09 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2023-12-17 00:00:00] [2023-12-20 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00] [2026-01-18 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00] [2023-12-11 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123] [2024-01-19 00:00:00, 2014-08-12 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00, 9999-12-31 00:00:00] [] [2001-11-06 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00] +174 32679 1 0 1 2023-12-11 2024-02-18 2024-06-30 2024-02-18 ? ? \N p - ? 考虑回你成绩符合系统目前在哪里呢打电话扩展 0 0 3.136600 68.116000 0.000000 40.123000 2014-08-12 00:00:00 2023-12-20 00:00:00 2023-12-19 00:00:00 2024-01-08 00:00:00 2024-06-30 12:01:02.123000 2023-12-17 00:00:00 2025-06-18 00:00:00 2023-12-14 00:00:00 [] [1, 1, 0, 0, 0] [] [0, 1, 0, 0, 1] [] [127, 127, 123] [0, -126, 7, 8, 1, -128, 115, 0] [] [32767, -32768, 32192, 17626, 245, -32768, 1, 32767, -17576, -8800] [1, -32768, 245, 25016, 15747] [21859, 25938, -5684, -6571, 7581, 32767, 29885, 15512] [32767, 0, 20968, 245, 1, -32768, 5297, 245, -6775, 5943] [375570785, 147483648, 1, -1, -2147483648, 0, 0, -1, -1586454330, 2147483647] [1, 1, 0, 1, -2147483648, 2109502439, -857508763, -1563386558] [] [-1101726455, -1, 32679, -1, -2147483648, 1292621940, 144796387, 147483648] [-1, -9223372036854775808, 0] [-1619688884, -1887489417733860792, 9223372036854775807, -9223372036854775808, 2308508656077072136, -1364763, 245, 1] [] [9223372036854775807, 1482902, 1, 147483648, 32679, 1, 245, 7556300, 926166, -7480907649352210250] [1, 0, 1, 0, 2, 2, -2, 0] [] [] [-704822696, -857280356, 7] [1.200000000000, -1.200000000000, 300.343000000000, 300.343000000000, 100.020000000000] [3.069300000000, 97.042300000000, 78.056400000000] [-1.200000000000, 100.020000000000, 1.200000000000] [1.200000000000, 300.343000000000, 99.027400000000] [-1.2000, 40.1230, 0.0000, 18.0376, 100.0200, 9.0950, 0.0000, 100.0200, 0.0000, 0.0000] [-1.2000, 1.2000, 1.2000] [100.1902, 6.1739, 300.3430, 300.3430, -1.2000, 1.2000, 300.3430, 6.1536, 1.2000, 71.0653] [82.0569, 40.1230, 28.0467, 100.0200, 11.1938, 30.1885, 100.0200, 100.0200] [1, 0, 1, 28, 100] [0, 300, 100] [100, 300, 25, 0, 300] [34, -1, -1, 26, 89, 300, 0, 1, -1, 100] ["", "肥肉有点忙伟泽采购总也宝贵代理商", "just", "tell", "?"] [] ["is on come his hey", "-", "-", "长时间晚上打", "双飞燕", "", "还有呢", "time there her is now i the like how of", "", "--"] [] ["p", "", "难缠", "万能不了浪费维修代玉懂吗烂漫硬件", "r", "参与不过现代接口阿里巴巴条款副本号发给王峰想开点", "look", "f"] ["-", "?", "something", "有你们", "been yes me", "相信", "a", "oh", "查收", "之处"] ["--", "the one from that got been good see back I'm", "got", "no", "the no"] [] [] ["b", "r", "k", "x", "y", "b", "n", "l", "n", "q"] [] ["f", "q", "d", "w", "n", "r", "i", "x", "q", "a"] [] [2023-12-17, 2023-01-15, 2023-12-14, 2019-09-17, 2023-12-11, 2023-01-15, 2025-02-18, 2023-12-20] [2025-02-17, 2024-01-31, 2023-12-11] [] \N [2023-12-15 00:00:00, 2016-05-28 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2013-03-02 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59] [2023-12-16 00:00:00, 2023-01-15 08:32:59, 2023-12-20 00:00:00, 2024-06-30 12:01:02, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02, 2023-12-19 00:00:00, 2024-08-03 13:08:30] [2023-12-10 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00] [] [2023-12-20 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00] [2023-12-16 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2023-12-16 00:00:00] \N [2025-06-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-14 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2023-12-13 00:00:00, 9999-12-31 00:00:00, 2023-12-14 00:00:00] [2023-12-12 00:00:00, 2023-01-15 08:32:59.123123, 2024-08-03 13:08:30] +176 \N 0 -2147483648 -1 2023-01-15 2024-02-18 2023-12-11 2024-01-31 -- 不至于前几天 -- ? 瑞丽 they think be okay all when did oh 0 1 \N 0.000000 71.179800 100.015300 2023-12-16 00:00:00 2025-06-18 00:00:00 2024-01-08 00:00:00 2026-01-18 00:00:00 2023-12-10 00:00:00 2023-12-18 00:00:00 2024-02-18 00:00:00 2012-12-17 00:00:00 [1, 0, 0, 0, 0, 1, 0, 0] [] [1, 1, 1, 1, 0] [0, 0, 1, 0, 0, 0, 1, 1] [20, -1, 0, 1, 0, 1, 6, -58] [-95, 71, 127, -74, 112, 127, 127, -42, 127, 0] [-1, 1, 70, -114, 74, 127, 6, 127, 1, -125] [51, 0, 45, 48, -74, 4, 57, 127] [-32701, -24750, 29655, 31021, -25782] [22421, 14733, 245, -1, 30044] [] [14863, 245, -32768] [] [1, 0, 1, 32679, 1598052053, -1, 147483648, -1470240236] [1494228749, -1, 147483648, -1652949734, 46444262, -1869881401, 179797554, 147483648, 263001664, 2147483647] [-2147483648, 147483648, 826128792, -1, 2147483647, 2147483647, 147483648, 147483648] [] [9223372036854775807, 147483648, -1, 848521172, -1526115299, -2743436233813120968, -9223372036854775808, 147483648, -5738536, -1] [-1, 9223372036854775807, 147483648, 0, 2692124868174726117, 4944011, 1, 1, -9223372036854775808, -9223372036854775808] [] [-1, -334586444, 0, 3, 1, -1, -2, 692656823, -2133894259, -1] [-2, -2, 2, -1, 1] [-2, 0, -1] [0, -1, 0, -2, -1, 0, 7, 1] [0.030100000000, 300.343000000000, 1.200000000000] [40.123000000000, 65.048500000000, -1.200000000000, 96.056000000000, 1.200000000000, 0.000000000000, -1.200000000000, 1.200000000000] [74.084100000000, 0.000000000000, 55.025300000000, 100.020000000000, 20.032200000000] [1.200000000000, 19.074900000000, 300.343000000000, 16.169500000000, 40.123000000000, 300.343000000000, 300.343000000000, 54.110600000000] [0.0000, 73.0807, 90.1976] [91.0616, 100.0200, 40.1230, 100.0200, 1.2000] [1.2000, 40.1230, -1.2000, 40.1230, 32.0488, 51.1754, -1.2000, 300.3430] [40.1230, 300.3430, 40.1230, 0.0000, -1.2000] \N [1, 1, 40, 17, 8, 100, 100, 1] [-1, 15, 100] [40, 300, -1, 40, 0, 88, 100, 40] ["总天数", "-", "开会比如说看重小计道理周鹏带来昨晚上有空吗磁带库", "x", "?", "飞扬", "think good are my out well think not i when", "--", "h", "right"] ["--", "销售对吧是你或", "授易", "not about be was yes if", "--", "联想", "--", "", "be can't think had it it if it's me i", "i"] ["--", "didn't", "所谓", "中晶", "", "新联惠", "支原体表示老同学呵呵呵", "hey"] ["对咱们金水区难缠里有", "it hey they don't", "操心出去给你听产品网通", "just want i on we what think back", "营销夏娜因为电子卡将那样升职办事处产生"] \N ["?", "been", "颜色产品事业部经理关闭", "安华", "--"] ["不错", "明珠", "?", "九州", "about would the now"] [] ["n", "o", "h", "l", "x"] \N [] ["l", "t", "q", "h", "m", "t", "o", "y"] [2014-08-12, 2024-01-19, 2024-02-18] [2023-12-17, 2024-01-17, 2027-01-09, 2023-12-14, 2023-12-19, 2025-02-17, 2023-12-09, 2023-12-17] [2014-08-12, 2023-12-20, 2023-12-11, 2024-01-31, 2026-02-18] [2023-12-15, 2024-08-03, 2024-07-01, 2023-12-18, 2024-02-18, 2014-08-12, 2025-02-18, 2024-01-17, 2024-07-01, 2023-12-12] [2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 2023-12-15 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2014-08-06 00:00:00] [2024-02-18 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02, 2025-06-18 00:00:00, 2026-01-18 00:00:00] [2023-01-15 08:32:59, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59] [9999-12-31 23:59:59, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00, 2024-08-03 13:08:30, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00] [2023-12-17 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00] [2023-12-18 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00] [2027-01-16 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2016-04-09 00:00:00, 2024-02-18 00:00:00] [2014-08-12 00:00:00, 2023-01-15 08:32:59.123123, 2014-08-12 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-18 00:00:00] [2023-12-20 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00] [2024-07-01 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2023-01-15 08:32:59.123123, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00] +177 764710132 0 1 32679 2023-12-10 2023-12-11 2023-12-10 2025-06-18 ? ? 很错下周前端 一辈子大学路海星这首幕墙千兆网卡天空 - 我不会玩心思贰仟生病气死故事客户文件一件 扩展名 1 0 1.200000 1.200000 0.000000 10.064000 2023-12-13 00:00:00 2026-01-18 00:00:00 2023-12-18 00:00:00 2024-06-30 12:01:02.123000 \N 2025-06-18 00:00:00 2023-12-11 00:00:00 2024-01-17 00:00:00 [0, 1, 1, 0, 0] [0, 0, 0] [1, 1, 1, 1, 0, 1, 1, 0, 0, 0] [1, 1, 0] [-87, 52, 6, 91, -89] [] [] [] [-4148, 28507, 2895, -9629, 32767] [32767, -1, -1, -8709, 0] [-22561, -9964, -29683, -1, 245, -27566, -11619, 9529, -11018, -1] [4888, 245, -5320] [0, 651415306, -2147483648] [] [1351216494, -2147483648, 314981834, 808619069, -1033555493, 2147483647, -1, 0, 1, 1641578072] [147483648, 0, 32679] [1, -1, 5283260724962199023, 0, 514837594] [] [1797613286, 7681347, -3547355926125721838, 582705224749553388, 32679, 245, 245, 6640628559476607909, 1883326652, 681693757] [-7307191623594691668, -9223372036854775808, -1, 2101533000, -1] [-1, -1, 6] [8, 1, -2, 1, 1256425263] [1, -1, 5] [0, 0, 9] [100.020000000000, -1.200000000000, 0.000000000000, 90.006500000000, -1.200000000000] [300.343000000000, 300.343000000000, 18.033300000000, -1.200000000000, 300.343000000000, 100.020000000000, 12.081700000000, 40.123000000000, 300.343000000000, 300.343000000000] [-1.200000000000, 300.343000000000, 1.200000000000] [40.123000000000, 15.127500000000, 1.200000000000, 23.004200000000, 36.142900000000] [0.0000, 6.0817, 0.0000, 4.1361, 0.0000] [-1.2000, 20.1772, -1.2000, 62.1769, 1.2000] [56.1550, 87.1283, 92.0554, 100.0200, 100.0200, 0.0000, 78.0559, 100.0200, 81.0172, 300.3430] [40.1230, 78.1117, 0.0000, 0.0000, 1.2000, 300.3430, 1.2000, 40.1230, 16.0295, 26.1592] [100, 300, 300, 300, 40, 0, 0, 1] [300, 300, 100] [-1, 40, 97, 0, 300, 300, -1, 100] [] \N ["if", "邮政", "she", "?", "大企业这家没有货不应该还向金科华云鹏", "失恋太多", "", ""] ["网景环境现实中限价知道了赛奥住宿费手下具体不了", "a", "神秘"] ["be okay", "now", "?"] \N ["something", "got this he's now who in out or look", "都是", "?", "can't"] ["学员群殴价值可选海川黄河科学你能看到忘不了", "what had say if", "", "really", "一看", "his", "还有呢不了", "女朋友", "--", "time they when had oh not i didn't with i"] ["--", "want", "赔钱一点翻译温总轻信手续满足洗液趋向性"] ["w", "u", "o", "m", "c", "f", "m", "v", "x", "d"] ["i", "n", "n", "n", "g"] [] ["v", "b", "y", "e", "j", "n", "w", "m", "w", "f"] [9999-12-31, 2025-06-18, 2025-02-18] [2025-06-18, 2023-12-16, 2026-02-18, 2023-12-18, 2025-06-18, 2027-01-16, 2023-12-12, 2023-12-15] [2023-12-13, 2024-02-18, 2024-01-19] [2023-12-20, 2023-12-14, 2023-12-10, 2027-01-09, 2023-12-20] [2025-02-17 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2008-09-16 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00] [2027-01-16 00:00:00, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2004-08-13 00:00:00] [2026-01-18 00:00:00, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2024-01-19 00:00:00, 2023-12-19 00:00:00] [2024-01-17 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [2024-01-08 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2023-12-16 00:00:00] [] [2023-12-19 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00] [] \N [2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2026-01-18 00:00:00] [2023-12-15 00:00:00, 2024-01-19 00:00:00, 2023-12-11 00:00:00] [2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-19 00:00:00, 2023-12-16 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] +178 \N 0 -901284649 -1815271120 2025-06-18 2024-08-03 2025-02-18 2024-01-19 - -- 冤枉 back a 建新开头过去翔龙宏运详情规定 ? that are yeah can't 1 1 100.020000 -1.200000 100.020000 1.200000 2023-12-09 00:00:00 2023-12-15 00:00:00 2006-09-10 00:00:00 2023-12-17 00:00:00 2023-12-15 00:00:00 2002-04-05 00:00:00 2024-01-09 00:00:00 2023-12-19 00:00:00 [0, 0, 0, 0, 0] [0, 1, 1, 1, 1, 0, 0, 0] [1, 1, 1, 1, 1, 0, 1, 0] [0, 1, 1, 0, 1, 1, 1, 1] [-56, -128, -1, -128, -1] [7, 25, -1, 33, -22, 1, -101, -38] [124, -1, 127, -1, 1] [-128, 86, 30, -128, -14, -39, 0, 0] [1, 1, 5376] [-24220, 0, -32768] [1869, -26559, 26608] [0, -15983, -14784, 0, 27645] [-2147483648, 0, -2147483648, -2147483648, -1644244760] [147483648, 147483648, 813312149, 1908535981, -2147483648] [606269452, 2147483647, 1701468993] [-1899914158, 1785177400, 32679, -1976407519, 2147483647, -1, 1029567873, -2147483648] [6493812205081819800, -1, -1, 1, 920690085] [147483648, -4232414396061238901, 0, -9223372036854775808, 147483648, -1045999, 245, 147483648, 0, 245] [-306097441, -5786815, 266159882397564295, -1136479627465849708, 1, -9223372036854775808, 9223372036854775807, -6578064587224227925] [32679, 7910373382266071905, -4804923797918325011, -1, 4295359030501797571] [4, 850694982, 9] [8, 1, 0] [-1, -1, 0, 2001355402, 8, -1, 1, -1, -2, -1] [1, -2, 1] [300.343000000000, 1.200000000000, 0.000000000000, 49.153100000000, 1.200000000000, 60.057000000000, 1.200000000000, 51.039100000000, 300.343000000000, 0.000000000000] [1.200000000000, 0.000000000000, -1.200000000000, 100.020000000000, 17.177300000000, 0.000000000000, 40.123000000000, 4.087400000000] [] [100.020000000000, 26.062600000000, 37.017300000000, 40.123000000000, 100.020000000000, 100.020000000000, 0.000000000000, 1.200000000000] [0.0000, 66.1689, 0.0000] [-1.2000, 300.3430, 53.1776, 40.1230, 9.0767, 27.1014, -1.2000, 300.3430, -1.2000, 0.0000] [100.0200, 100.0200, 1.2000, 100.0200, 1.2000, 1.2000, 40.1230, 19.1433, 1.2000, 1.2000] [] [76, 40, 1, 1, 26, 25, 22, 300, 66, 300] [-1, 300, 17, 74, 300] [58, 300, 300] [40, 300, 11, 1, 54, -1, 100, 40, 71, 16] ["-", "something know got they so who not mean i back", "yes on you in have yes or would it with", "整体", "明天会更好", "-", "out come that's mean something would was just", "?"] ["time", "这几太难人见失误真正拼搏农村王先生", "群号", "right and no", "?", "y", "the", "o"] ["格飞拜访等级九州做人", "for", "最重要"] ["back it's look just", "--", "以下", "谦虚办公室光纤做到以下珊瑚许愿树油漆不想", "?"] ["煤气", "on this well okay we you her", "做完", "--", "l"] ["didn't", "okay", "r"] ["驱动器以为停机珊瑚喝口见到广告词质量", "济南词语中成认为金辉懂吗", "供货交到不在锐捷信任一共彼此辅佐"] ["something I'm well okay want him if just", "质保要不然合作", "", "实力", "?"] [] ["q", "b", "s", "c", "u", "a", "v", "s", "r", "m"] ["d", "f", "w", "k", "d", "u", "l", "h"] ["h", "k", "y", "i", "l"] [2023-12-09, 2023-12-16, 2023-12-17, 2014-08-12, 2023-12-20] [2025-02-17, 2023-12-19, 2023-12-11, 2024-01-09, 2023-12-11, 9999-12-31, 9999-12-31, 2025-06-18] [2026-02-18, 2023-12-11, 2024-01-09, 2024-01-09, 9999-12-31, 2023-12-14, 2023-01-15, 2023-01-15, 2023-12-10, 2027-01-16] [] [2024-01-31 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00] [2024-01-19 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00] [9999-12-31 23:59:59, 2023-12-16 00:00:00, 2024-02-18 00:00:00] [] [2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00] [2023-12-16 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2011-11-26 00:00:00, 2023-12-18 00:00:00, 2024-01-17 00:00:00] [9999-12-31 23:59:59, 2014-08-12 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00] [2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00] +179 1 600673536 99301268 1853904993 2000-11-12 2025-02-17 2023-12-10 2027-01-16 尝试 得劲 没有货 him can now come we really have 小庞作为供应了吗失误山下到帐离职万邦供销大厦 can my look d of 1 0 1.200000 23.182100 300.343000 -1.200000 2018-10-16 00:00:00 2023-12-13 00:00:00 2025-06-18 00:00:00 2024-01-09 00:00:00 2023-12-19 00:00:00 2026-01-18 00:00:00 2023-12-11 00:00:00 2024-01-09 00:00:00 [0, 0, 0, 0, 0, 1, 0, 0] \N [] [0, 0, 0, 0, 1, 0, 0, 0] [127, -1, 0] [1, 0, -111] [127, -128, -128] [] \N [1, -2485, 1, 0, -9896, 1, -32768, 1, -1, 11177] [-10853, 245, 0] [] [32679, 147483648, -1576393469] [1, 0, -1088227774, -1401681845, 1051969375] [-282571556, 1430204271, 1896479539, 32679, -1574220075] [0, 2147483647, 32679, 1330228750, 2001792575] [245, -5141459307900658146, -757649338, 1, -1] [] [-5557435022307689039, -8344676721224454901, 1939408830] [5705899210323878514, -9223372036854775808, 32679, 147483648, 5455723, 7641140204042831603, 3627270, -9223372036854775808] [4, -2, 0, 0, -1, 1, 1, -2, 7, 1] [7, -2, -878741772, -1812005429, -2] [1, -1, 1214426291, 6, -1, -24262090, -1, 1, 1, -1] [] [1.200000000000, 18.123900000000, 12.148600000000, 26.049300000000, 1.200000000000, 95.079000000000, 100.020000000000, 300.343000000000] [1.200000000000, 100.020000000000, 100.020000000000, 100.020000000000, -1.200000000000, 16.134500000000, 1.200000000000, 100.020000000000] [-1.200000000000, 100.020000000000, 1.200000000000, 300.343000000000, 2.162900000000, 1.200000000000, 27.170800000000, 40.123000000000] [81.025900000000, 100.028300000000, 40.123000000000, 300.343000000000, 14.080800000000] [] [50.0989, 300.3430, 1.2000] [300.3430, 60.0038, 0.0000, 6.0538, 100.0200, 1.2000, 28.0780, 0.0000, 0.0000, 300.3430] [100.0200, 300.3430, -1.2000, 40.1230, 40.1230, 94.0984, 100.0200, 100.0200, 1.2000, 1.2000] [] \N [-1, 1, 99] [30, 100, 1, -1, 40, 1, 300, 70] \N ["", "可以", "生活而来写上去", "had", "-", "功能反正那么大连接线谈判乔经理备用金贸易", "mean she what know and something were got that's", "共同北站效果授狗长光相符中欧重要前辈未定"] ["--", "that's", "", "准备", "-", "that", "王佩丽陈老师这次拥有", "空间冗余标志新买送货昨天军港", "--", "b"] ["记得", "梅捷", "mean", "?", "i", "had they out he's tell when in okay it his", "", "王哥赵经理", "恢复", "like look back because for can will"] ["", "on", "--", "m", "been then like", "有个", "?", "--", "I'll", "-"] [] ["股份有限公司", "-", "hey say", "", "your her is do know", "?", "my", "五万条", "it", "石龙"] [] ["g", "g", "x"] [] ["g", "w", "u", "s", "u", "j", "i", "x"] [] [9999-12-31, 2024-01-17, 2024-01-31, 2025-06-18, 2024-06-30] [] [] [2024-01-17, 2024-08-03, 9999-12-31, 2025-06-18, 2027-01-16, 2025-06-18, 2024-02-18, 2023-12-19] [2023-01-15 08:32:59, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2016-06-27 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59] [2023-12-11 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00] [2023-12-18 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2024-01-17 00:00:00, 2023-12-16 00:00:00, 2023-12-11 00:00:00, 2024-01-09 00:00:00] [2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-01-19 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00] [2024-01-09 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-11 00:00:00, 2023-12-13 00:00:00] [] \N [2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2024-07-01 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 2023-12-19 00:00:00] [2023-12-09 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00] [9999-12-31 00:00:00, 2023-12-17 00:00:00, 9999-12-31 23:59:59] +180 147483648 2147483647 1701498337 1845222503 9999-12-31 2024-07-01 2023-12-12 2023-12-14 \N ? \N now here was 出纳 农业跟我 下午付出一下不说 \N 1 300.343000 40.123000 77.038300 300.343000 2024-01-09 00:00:00 2024-06-30 12:01:02.123000 2024-01-09 00:00:00 2023-12-18 00:00:00 2023-12-10 00:00:00 2024-01-09 00:00:00 2026-01-18 00:00:00 2024-01-17 00:00:00 \N [1, 1, 0] [] [1, 0, 1, 0, 1] [15, 1, -128, -128, -1, -32, -1, 0] [] [-1, -1, 0, 1, 1, 5, 1, -128, -108, 0] [122, -1, -18, -1, -128, 2, 1, -128] [1, 6419, -3722] [32767, -23948, 19263, -29209, -25917, -21208, 13028, -32768] [1, -1319, -8876] [8881, 9387, 23430, 23660, -1, -1, 9328, 1] [2147483647, 2147483647, 2030849112, 712857202, 535301497] \N [2077180394, 2147483647, 32679, 641750768, 32679, 2147483647, 0, -1, 296561777, 1890664270] [] [0, 3463034415006501060, 32679, 0, -9223372036854775808] \N [-9223372036854775808, -3508777114004774580, -1329812942, 0, -2056754792, 32679, 147483648, 0, 0, 0] [-1, 9223372036854775807, 1, -8750905742118083700, -9223372036854775808, 8665359932434364444, 32679, 32679, 32679, -4939996] [1, 1647213326, 7, 0, -1, -2, 1, 0] [8, 1, 0, -1320933727, -439294297] [1, -2, -2, -1, -2] [206248873, -1837574718, -1, 1, -1, 1139628435, -65832080, 0, -1, -1911585570] [-1.200000000000, 1.200000000000, -1.200000000000] \N [39.197900000000, 40.123000000000, 1.200000000000, 11.058200000000, 300.343000000000] [100.020000000000, 100.020000000000, 0.000000000000, -1.200000000000, 1.200000000000] [100.0200, 32.1926, 100.0200] \N [59.1942, 100.0200, 40.1230, 93.0727, 93.1047, 40.1230, 300.3430, 300.3430] [40.1230, 300.3430, 1.2000, 1.2000, 0.0000] [300, 21, 0, 37, 40, 100, 13, 40, 100, 79] [1, 18, 60, 0, 100, 0, 70, 40, 40, 1] [40, -1, 100] [89, 40, 100, 0, 59, 0, -1, 100] ["拆机很大事业晓得委托收款周期看着", "圆圆春子开机", "why who hey"] ["订单大键盘吴静通话一块小欣服务站集成三大我不", "what it going because in something he's here the could", "--"] ["", "?", "like at had don't we in as"] [] ["", "b", "陪着经常屠龙记彭巧语我们名次分别数码港英联"] ["?", "have it's was he how know don't", "你吃法"] ["here well for they don't know we can", "查收", "机器办完", "", "--", "was", "冀海潮点事普腾企业", "be but of but so is"] ["有个祺祥摄像头周口", "网站一条线专用你不工作吗", "", "", "x"] ["q", "n", "d"] ["i", "v", "f", "m", "c", "w", "t", "p", "p", "w"] [] ["n", "m", "v", "o", "a", "q", "c", "o"] [9999-12-31, 2023-12-14, 2027-01-09] [2023-12-17, 2024-01-17, 2026-02-18, 9999-12-31, 2024-01-31, 9999-12-31, 2025-06-18, 2025-06-18, 2023-12-13, 2027-01-16] [] [2023-12-20, 2024-08-03, 2026-02-18, 2024-01-09, 2026-01-18, 2025-02-18, 2023-12-14, 2024-02-18, 2024-06-30, 2025-02-17] [2023-12-12 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59, 2024-07-01 00:00:00, 2014-08-12 00:00:00, 2002-12-03 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00] [] [2024-08-03 13:08:30, 2024-06-30 12:01:02, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00, 2023-12-13 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00] [2026-01-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00, 2023-12-12 00:00:00, 2023-12-14 00:00:00] [2023-12-13 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2024-01-08 00:00:00] [9999-12-31 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00] [2023-01-15 08:32:59.123000, 2023-12-13 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00] [9999-12-31 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-18 00:00:00] [2024-01-08 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [] [] +182 0 -2147483648 -1940344848 147483648 9999-12-31 2027-01-16 2014-08-12 2024-01-09 赵燕燕 投缘 can't okay on tell ok can't from of okay 高性能 -- tell 0 0 100.020000 5.087800 1.200000 42.057600 2024-02-18 00:00:00 2023-12-17 00:00:00 2024-07-01 00:00:00 2024-01-17 00:00:00 2023-12-15 00:00:00 2023-12-17 00:00:00 2014-08-12 00:00:00 2024-02-18 00:00:00 [1, 0, 0, 0, 1] [1, 0, 1] [] [1, 1, 0, 0, 1, 1, 0, 0, 1, 1] [-128, -32, -1, 0, 74, 0, 1, -47] [] [] [82, -72, -127] \N [3824, -1149, -11718, 245, -7799, -1, 22466, -29304] [] [-1, 8949, -31988, -322, -24939, -1, 8, 15519] \N [-1126387533, -1464191920, 1, 147483648, -2147483648, 32679, 716613336, 147483648, -445622326, -1] [147483648, 819427139, 0, -1, 1, 147483648, 261489048, 0] [2147483647, -1905692366, -13673877, -2058022004, 1663331193] [1090465294, -5672606109309784640, 8482945196609940566, -9223372036854775808, 8273965365692418885, -1841434913, 32679, 9223372036854775807, -2812654162934644128, 1] [147483648, 32679, 1, 0, 9223372036854775807, -1851558946, -1553039437, 1] [-916730298958776882, 32679, 147483648, -1, -1, 9223372036854775807, 0, 2674592] [] [1, 1, -1, 717872988, 1] [-2, -1848159337, 1839907201] [] [] [] [40.123000000000, 300.343000000000, -1.200000000000, 100.020000000000, -1.200000000000, 40.123000000000, 0.000000000000, 300.343000000000, -1.200000000000, 100.020000000000] [0.000000000000, 100.020000000000, 4.041000000000, 300.343000000000, -1.200000000000, 0.000000000000, 40.123000000000, 40.123000000000] [-1.200000000000, 300.343000000000, 40.123000000000, 100.020000000000, -1.200000000000, 100.020000000000, 0.000000000000, -1.200000000000, 100.020000000000, 1.200000000000] [74.0110, 88.1796, 40.1230, 100.0200, 40.1230, 28.0701, 1.2000, 81.1561, 100.0200, 100.0200] [40.1230, 18.0785, 300.3430] [-1.2000, 300.3430, -1.2000] [51.1045, 67.0615, 96.0603, 95.0588, 100.0200, -1.2000, 300.3430, 66.1976, 300.3430, 300.3430] [0, 77, -1] [300, 300, 11] [] [31, 0, 50, 1, 40, 0, 28, 0, 100, 1] ["移动国际企业国企中层经理多少钱", "--", "?", "发放天气华阳凄楚规格伟博到帐光驱雷林查证", "-", "can't they get don't they then want", "为二路", "when got a why yeah him", "?", "r"] ["as", "t", ""] ["--", "明显许昌", "that"] ["", "弯弯的月亮", "right get all"] \N ["多少信息后颈华留言", "been", "果断", "?", "o", "要求", "-", "第一次"] ["have", "-", "-", "几下一千多明威交局法律爱情吉林省去掉软件", "", "was with what about we", "合作过", "?", "l", "that's or are no good do"] ["did", "春子", "could okay it now"] \N \N ["o", "j", "w"] ["t", "q", "q", "p", "q", "z", "g", "p"] [2023-12-19, 2023-12-10, 2023-12-19] [2024-02-18, 2027-01-09, 2024-02-18, 2023-12-13, 2024-08-03, 2024-07-01, 2023-12-20, 2024-02-18, 2025-02-17, 2023-12-18] [2024-01-09, 2024-06-30, 2024-08-03, 2023-12-13, 2025-02-18, 2023-12-18, 2023-12-18, 2025-02-17] [2024-01-17, 9999-12-31, 9999-12-31] [] [2023-12-20 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2002-09-19 00:00:00, 2024-01-31 00:00:00, 2023-12-16 00:00:00, 2023-12-13 00:00:00] [2023-12-17 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2027-01-16 00:00:00, 2002-03-03 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2024-06-30 12:01:02, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2023-01-15 08:32:59] \N [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00] [] [2023-12-10 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00] [2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00] [9999-12-31 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00, 2023-12-14 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00] [2024-01-17 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [] +187 -1 0 1 176852954 2024-01-19 2025-06-18 2023-12-16 2024-01-08 ? \N k come - 不支持 感触这一块承诺李金才烟火一栏汉化补丁部队相识有人 time here really yeah he did your 0 0 94.182700 100.020000 65.179400 0.000000 2014-08-12 00:00:00 2023-12-17 00:00:00 2023-12-13 00:00:00 2014-08-12 00:00:00 2027-01-09 00:00:00 2023-12-14 00:00:00 2023-12-09 00:00:00 2024-02-18 00:00:00 [0, 0, 0] [0, 0, 0] [0, 1, 1, 1, 1, 0, 1, 1, 0, 1] [0, 0, 1, 1, 1, 0, 0, 0, 0, 1] [0, -128, -128, 127, 8, 127, 81, -76, 1, -1] \N [-40, 44, 0, 9, -113] [127, 36, -57, 0, -1, 9, -128, -12] [-26350, 18628, 5813, -16137, 8600] [] [245, 21028, 1363] [-13440, -11361, 3451, -12496, -23474] \N [-1, -1, 2147483647, 965312598, 0, 375216421, -1, 2147483647] [-1926841796, 32679, -1669166297, 0, 0] [147483648, 0, 196411380, 0, -52963728, 32679, 0, 1, 147483648, -222597702] [-9223372036854775808, 1, 32679, -2008851019, 8717503746413198454, 2997508, -8828456212235730011, 0] [-1, 32679, 7989205, 1, -254322] [] [433910, 32679, -9223372036854775808, -9223372036854775808, 32679] [-2, 9, 8, 1, -1, 6, 1, 1] [0, -1, 1] [1049708097, -1, -1] [-1, -1711744526, -1, 3, -1] [72.095400000000, 43.110300000000, 98.042400000000, 300.343000000000, 300.343000000000, 39.116700000000, 97.045000000000, -1.200000000000] [300.343000000000, 57.114200000000, 67.124000000000, 1.200000000000, -1.200000000000] [] [1.200000000000, -1.200000000000, 40.033800000000] [57.1446, 26.1615, 300.3430, 300.3430, -1.2000, 100.0200, 1.2000, 100.0200] [85.0623, 100.0200, 40.1230] [-1.2000, 100.0200, 1.2000, 36.1053, 0.0000, 40.1230, 40.1230, -1.2000] [100.0200, 68.1462, 15.0992, -1.2000, 0.0000, 3.1240, 0.0000, -1.2000, 89.1313, 40.1230] [300, -1, -1, 1, 100, 89, 300, 40, 40, 42] [1, 100, 300] [40, 56, 300] [300, 1, 100] ["?", "好人坏人提成点安全大忙人华北区", "k", "be", "--", "一新", "at", "", "-", "j"] ["很细那个", "--", "go your mean", "一种", "just", "-", "严格初五想不到终究售后查证交到提出早点留一下", "but"] [] ["性能", "a", "-", "接收", "琵琶语"] ["?", "yeah", "", "--", "县城付出平常落实后面", "--", "we", "l", "f", "p"] [] ["?", "had", "?", "经三路沐浴未定宇瑞圣诞节", "she", "some okay we got if know", "k", "w", "would when okay hey have the", "看待"] ["?", "上演", "跟得上报名", "out not so yes tell", "with she and for as go he's there this now", "that's of my from think that's do a he", "do", ""] [] ["l", "g", "g"] [] ["i", "s", "s", "s", "w", "s", "n", "i", "d", "b"] [2024-01-08, 2023-12-15, 2024-01-17, 2024-01-31, 2023-12-10, 9999-12-31, 2024-02-18, 2023-12-17] [2023-12-15, 2023-12-17, 2024-02-18, 2023-12-20, 2024-01-31, 2024-08-03, 2003-12-26, 2023-12-09] [2023-12-16, 2024-08-03, 2026-02-18] [2024-01-17, 2023-12-16, 2026-01-18, 2023-12-14, 2024-01-19, 2015-08-07, 2025-06-18, 9999-12-31, 2026-01-18, 2024-01-31] [2027-01-09 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 2024-01-08 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00] [2024-01-31 00:00:00, 2027-01-16 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00] [] [2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] \N [9999-12-31 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2025-02-18 00:00:00] [2024-08-03 13:08:30, 2023-12-15 00:00:00, 2023-12-17 00:00:00] [2016-02-15 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-18 00:00:00, 2023-12-10 00:00:00, 2023-12-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2023-12-11 00:00:00, 2023-12-16 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2023-12-10 00:00:00] [2024-08-03 13:08:30, 2014-08-12 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00] [2025-02-17 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] +188 1 \N 1 -1549908219 2023-12-15 2026-01-18 2027-01-16 2024-08-03 明星每次被授予春节晚上有数以为你红颜照面词句 \N 独立 新日射 you're out out time right something I'm right were ? 整体交货下周合计自信今天焕然感悟 - \N 0 11.146900 \N 40.123000 3.084000 2026-01-18 00:00:00 2023-12-12 00:00:00 2024-06-30 12:01:02.123000 2027-01-09 00:00:00 2025-02-17 00:00:00 2025-02-17 00:00:00 2027-01-16 00:00:00 2024-01-31 00:00:00 [0, 1, 1, 0, 1] [1, 1, 1, 0, 0, 0, 1, 1] [1, 0, 1, 1, 1, 1, 0, 0, 0, 1] [1, 1, 0, 0, 1, 1, 1, 0, 1, 1] [] [-128, 23, 0, 127, 8, 1, 1, -32, 0, 0] [-51, -128, 127, -1, 118, 127, 37, 0, -128, -82] [31, 1, 105] [-22770, 0, -26655, 0, -14310, -4185, 1, 5146] [0, -21421, 17446, 28317, 1, -32768, 32767, -12845, 245, 23900] [] [-3436, 32767, 1, -32768, 1] [1076639599, 482734659, 0] [32679, -562049582, -1] [0, -575245063, -2147483648, 0, 2147483647, -634246547, 2147483647, 774683184] [-2114946740, 1606284765, 2147483647] [-1, 32679, 147483648, 9223372036854775807, 147483648, -9223372036854775808, 9223372036854775807, -9223372036854775808, 1360805451449213881, 306875512] [7866142, 1, -1, -2579526178227231664, 147483648] [0, -9223372036854775808, -9223372036854775808] [] [0, 7, 0, 1, -1] [1, -2, -2] [-1, -1, 6, -1107248546, 9] [-2, 0, -1, -2, -1, -1, 0, 1] [40.123000000000, -1.200000000000, -1.200000000000] [68.157400000000, 100.020000000000, 34.052300000000, 300.343000000000, 0.000000000000] [300.343000000000, 27.121900000000, 100.020000000000, 300.343000000000, 1.200000000000, 0.000000000000, 51.124800000000, 300.343000000000] [40.123000000000, 84.183900000000, 40.123000000000, -1.200000000000, 40.123000000000, 0.000000000000, 7.173900000000, 74.058900000000] [28.1653, 40.0550, 300.3430, 1.2000, 0.0000, 100.0200, 300.3430, 64.1866] \N [] [300.3430, 85.1788, -1.2000] [13, -1, 0] [35, -1, 50] [27, 0, -1, -1, 0] [300, 1, -1] ["先达低配废话解难等等我刚来呀", "-", "甲方", "m", "--"] [] ["-", "?", "用户"] ["at out now in see I'm going ok", "--", "now", "简介", "最起码"] ["?", "from one him some didn't then really", "对了", "because", "通过", "?", "-", "on there will", "广场供应刚刚出去双机热备方案来访键鼠", "不好"] ["?", "报价格", ""] ["瑞信每次群殴面包满意真正来说佳成值得全程", "about but who", "赞美诗", "转移", "节点注明凄楚连接线人往高处当地", "艾蕊商人周经理接触攻防思科", "--", "一想", "then no then", "on good me time with it's"] ["商厦有谁导演你问屠龙记电话本行踪有种梦想又去", "-", "s", "back are would hey no tell want well as", "--", "", "--", "", "will", "for"] ["m", "p", "z"] ["a", "v", "p"] ["j", "d", "t", "h", "a", "l", "k", "f"] ["t", "d", "n", "f", "q", "c", "v", "n"] [2023-12-20, 2023-12-15, 9999-12-31, 2025-02-17, 2023-12-17] [2023-12-10, 2023-01-15, 2024-01-09, 2025-02-17, 2014-08-12, 2023-12-13, 2025-06-18, 2023-12-10, 2023-12-12, 9999-12-31] [2023-12-12, 2024-06-30, 2023-12-14, 2023-12-20, 2023-12-16] [9999-12-31, 2023-12-20, 2024-07-01, 2025-06-18, 2024-08-03, 2023-12-19, 2023-12-18, 2023-12-09] [2023-12-16 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59, 2023-01-15 08:32:59, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59, 2014-08-12 00:00:00, 2023-12-18 00:00:00] [2024-01-19 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2019-07-12 00:00:00, 2026-01-18 00:00:00] [] [2023-12-14 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00, 2003-06-16 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2023-12-11 00:00:00, 2024-06-30 12:01:02, 2025-02-18 00:00:00] [2023-12-12 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00, 2016-12-18 00:00:00, 2023-12-13 00:00:00] [2023-01-15 08:32:59.123000, 9999-12-31 23:59:59, 2024-01-17 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00] [2023-12-13 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2025-02-18 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2014-08-12 00:00:00, 2024-01-31 00:00:00] [2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-10 00:00:00, 2023-12-19 00:00:00, 2027-01-16 00:00:00, 2024-06-30 12:01:02.123000] [] [2025-06-18 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00] [] +190 -600182479 1373559114 1464856275 -1608500710 2027-01-16 2024-07-01 2014-08-12 2023-12-17 h that what - q -- - he's 0 1 -1.200000 0.000000 61.034600 300.343000 9999-12-31 23:59:59 2023-12-12 00:00:00 2024-02-18 00:00:00 2013-09-18 00:00:00 \N 2023-12-20 00:00:00 2024-01-19 00:00:00 2027-01-16 00:00:00 [1, 1, 0, 0, 0, 1, 1, 1] [1, 0, 0, 0, 1, 0, 0, 0, 1, 0] [0, 0, 0, 1, 1] [0, 0, 1] [] [127, -1, 8, 127, -2, -40, 0, 1] [-1, 1, 0, 0, 127, 83, -1, 1] [-113, -1, -128, -43, -35] [-5548, 13522, 1, 1, -32768, -12229, 2061, -26236] [32767, 14810, -32768, -18090, -13365] [-15916, 20469, -30901] [-32768, 245, 5064, -1, 0, 15489, -1, 25038] [1391329452, 251211814, -842743302, -2147483648, 0, -817334580, 908161740, -1624160060, 347460746, -2081999208] [0, 147483648, 2147483647, 1958185567, -1] [1675718618, 434310170, 1] [-1, 1199934942, 2144384373] [] [-696078427733358469, 32679, 32679] [0, -1, 245, 245, -9223372036854775808] [147483648, 5265367447999401328, 0, 245, -1, 245, 0, 32679, 32679, 1695640547] [] [1, -464179089, 1] [] [7, -2, 1, 0, -2] [300.343000000000, -1.200000000000, 6.074200000000, 12.118900000000, 300.343000000000, 11.104300000000, 45.197700000000, 34.091900000000, -1.200000000000, 1.200000000000] [100.020000000000, 25.154500000000, 88.104300000000] [21.149400000000, 300.343000000000, 40.123000000000] [2.185500000000, 300.343000000000, -1.200000000000, 100.020000000000, 28.070100000000, 1.200000000000, 300.343000000000, 6.068000000000, 0.000000000000, 100.020000000000] [1.2000, 85.0485, 100.0200, 100.0200, 0.0000, 300.3430, 100.0200, 1.2000] [84.1653, 100.0200, 300.3430, 100.0200, 1.2000] [70.1857, 98.0310, 40.1230, 300.3430, 75.0103, 40.1230, 5.0441, 40.1230] [0.0000, 40.1230, 100.0200, 94.1898, 0.0000, 100.0200, 0.0000, 100.0200, 25.0187, -1.2000] [100, 40, 0, 72, 42] \N [83, 40, 1] [1, -1, 0, -1, 31, 45, 0, 19] ["back why didn't of as with mean would", "i", "分数航道传真富通中小型", "-", "w"] ["-", "what been could oh how be", "you not hey at"] ["didn't", "v", "上面送人但是"] ["-", "暂时", "now you're got there this on well", "come", ""] [] [] ["多少", "oh just that's that's your time him like", "", "will want when hey", "not and are all we just that's me oh", "?", "I'll", "--", "本来", "高兴之日起"] ["v", "-", "s", "come", "害怕"] [] ["q", "d", "h", "a", "t", "y", "o", "g", "b", "y"] ["x", "e", "s"] ["z", "l", "t", "q", "n", "x", "q", "q", "a", "s"] [9999-12-31, 2023-12-13, 2023-12-13, 2024-01-19, 2024-01-19] [] [2023-12-11, 2026-02-18, 2023-12-09, 2024-01-19, 2014-08-12, 2024-01-17, 2023-12-17, 2024-01-09] [2023-12-11, 2023-12-10, 2024-02-18] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00] [2014-08-12 00:00:00, 2024-02-18 00:00:00, 2014-09-24 00:00:00] [2025-02-17 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [2023-12-12 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00] [2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00] [2024-01-08 00:00:00, 2023-12-18 00:00:00, 2024-01-08 00:00:00] [2024-01-17 00:00:00, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2012-03-10 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-01-31 00:00:00, 2026-02-18 00:00:00] \N [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2023-12-17 00:00:00] [2026-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2011-01-11 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00] [2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2023-12-14 00:00:00, 2023-12-18 00:00:00] +191 32679 979747482 32679 -2147483648 2023-12-14 2027-01-16 2023-01-15 2023-12-13 \N for can \N 顺河路 天浩一直玖佰多多祺祥美好冤枉 新天智能 系诶邪恶 \N 1 40.123000 -1.200000 300.343000 300.343000 2025-02-17 00:00:00 2025-06-18 00:00:00 2025-02-18 00:00:00 2025-06-18 00:00:00 2027-01-09 00:00:00 2024-01-17 00:00:00 2024-01-09 00:00:00 2024-06-30 12:01:02.123000 [0, 0, 0, 0, 0, 0, 0, 1] [0, 1, 0, 0, 1, 0, 0, 0, 0, 1] [1, 0, 1] [0, 1, 0] [] \N [-7, 1, 127, 1, -128, 8, 127, -1, 0, 91] [0, -14, -103, 0, -1, 41, -128, 4] \N \N [14421, 16741, 24218, 0, 26210, 30294, 0, -32768, 0, -24795] [0, -2032, 0, -32768, 1, 0, -4616, 22525] [2147483647, -82749477, 2147483647, -149561306, 2147483647] [-206746781, -551513842, 1, 0, 2147483647, 2046455740, 2147483647, 552450569, -1, 881445997] [239747301, -1923954761, 1, 1604793746, -1318707145, 1096370736, -1829731211, 32679] [-1024442174, -1686984893, 0, 800037945, 0] \N [245, -2170349088370749417, 32679] [3509254427279481501, 32679, 147483648, 147483648, 2039821423] [3559098184515635815, 147483648, 1, 5284913789371911585, -9132550310062655175, -1, -9223372036854775808, -1, -9223372036854775808, 1] \N [1, 1, 3, 0, 0, 1, 0, -2] [-1, -2, 0] [0, 1070890685, 0, 0, -2, 0, -2, 0] [61.059500000000, -1.200000000000, 43.017900000000, 0.000000000000, 40.123000000000, 18.053000000000, 100.020000000000, 1.200000000000, 0.000000000000, 100.020000000000] [100.020000000000, 300.343000000000, -1.200000000000, 100.020000000000, 69.095300000000, 100.020000000000, 97.178400000000, 47.129600000000] [16.135900000000, -1.200000000000, 51.119800000000, 96.028800000000, 0.000000000000, 42.136800000000, 100.020000000000, 41.197900000000] [0.000000000000, 0.000000000000, 100.020000000000, -1.200000000000, -1.200000000000, 15.186400000000, 21.115500000000, -1.200000000000] [0.0000, 300.3430, 25.0014] [] [40.1230, 0.0000, 100.0200] [100.0503, 0.0000, 54.0578, 100.0200, 0.0000, 1.2000, 0.0000, 300.3430] [1, 40, 0, 0, 0, 0, 100, 40, 87, 100] [0, 40, 25, 40, 29] [100, 1, 3, 0, 1, 40, 0, 1] [100, 40, -1, 1, 100, 300, 1, 40, 300, 100] ["弄好贸易通这不是", "-", "", "?", "or", "up yes she want want out", "did", "大恒叁仟三洋"] ["记下", "营销", "from as not the really got see", "", "金水区又出元转到必要遥控器"] ["标准心事人生接近艺术那几手下有项目里有", "天成苑", "--", "or", "mean"] ["think like something he not all well right about that's", "-", "back about I'm to get at your"] ["I'll have some got I'm they if and", "年华天津", "--", "", "", "医院", "广通同时", "--", "time", "oh just"] ["", "--", "国务卿", "at me this some", "--", "接近牡丹商联吃饭了风泽园聊天对我好卖", "--", "", "?", "?"] [] ["-", "t", "then", "that", "be a", "yeah of", "was", "for"] ["q", "a", "h"] ["u", "d", "j", "s", "i", "p", "j", "v"] ["k", "u", "q", "h", "m", "b", "k", "p"] ["a", "m", "a"] [2026-02-18, 2024-01-09, 2023-12-11, 2023-12-17, 2026-01-18] [2024-02-18, 2027-01-16, 2024-01-08, 2024-01-08, 2026-02-18, 2023-12-15, 2024-01-09, 2025-06-18] [] [2027-01-16, 2024-02-18, 2024-02-18, 2024-08-03, 2026-02-18, 2024-02-18, 2023-12-16, 2023-12-10, 2026-02-18, 2024-08-03] [2023-12-20 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2018-06-09 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00] [2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2009-01-13 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00] [] [] \N [2023-12-16 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00] [] [2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2023-12-13 00:00:00] [2023-12-12 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123123] [2023-12-20 00:00:00, 2024-08-03 13:08:30, 2024-06-30 12:01:02.123000] [2024-08-03 13:08:30, 2027-01-09 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-19 00:00:00] [] +192 1859078425 323291828 -949268860 -2147483648 2023-12-14 2023-12-09 2023-12-14 2024-01-08 中龙 客气 the v ? she come will been when been well like would go could ? \N 1 300.343000 \N 1.200000 -1.200000 2023-12-12 00:00:00 2027-01-16 00:00:00 2024-01-19 00:00:00 2026-01-18 00:00:00 2024-07-01 00:00:00 2025-06-18 00:00:00 2024-01-09 00:00:00 2024-08-03 13:08:30 [] [0, 1, 1, 1, 1, 0, 0, 0, 1, 0] [0, 1, 0, 1, 1] [0, 0, 1, 0, 1] [1, 6, 1, 0, 127, 4, -128, 2] [127, 61, 127, -1, -1, 8, 1, 77, 1, 0] [0, -1, -12, 127, -11, 7, -18, -59] [] [] [] [] [] [] [-952906892, -964820729, 247713720] [147483648, -745818371, 1423821213, 278059209, 1, 1511060171, 1, -1598914812] [-1918868994, 1085226477, -2147483648] [-4242185, 0, 9223372036854775807, -533742, 3583464858669708267, 4784054, -7702944723466913535, 245, 245, 6431134] [1, 1, 344636, -9223372036854775808, -1, 245, 1, 147483648] [2069446481, 9223372036854775807, -6602430345243326883, 32679, -4714249766188948062] [147483648, -7421323657845060392, 267469629, 9223372036854775807, -1, 0, -1, 32679] [675281299, 1, -1350424062] [-1971966065, 1109202181, 9, 3, 1, -2, 4, -1, 8, -2] [1966218870, -2, 1, 0, 697520066, -2, 5, 1, 0, -2] [0, 0, 1, -1583188048, -743093454, 7, -1, -1157967072, -1, -1] [300.343000000000, -1.200000000000, 1.200000000000, 0.000000000000, 98.131300000000, 15.041500000000, 0.000000000000, 55.064000000000] [] [] [100.020000000000, 1.200000000000, 100.020000000000, -1.200000000000, 300.343000000000, 0.000000000000, 21.199900000000, 40.123000000000, 100.020000000000, 10.121500000000] [100.0200, 1.2000, 40.1230, 75.1012, -1.2000, 1.2000, 74.1615, 100.0200, 19.0556, -1.2000] [300.3430, 45.1554, 69.1171, 1.2000, 1.2000] [35.1846, 1.2000, 300.3430] [0.0000, 44.0656, -1.2000] [100, 100, 40, -1, 45, 40, 1, 100] [37, 7, 300] [40, 55, 69, -1, 76, 73, 44, 300, 300, 61] [6, 0, 300, 1, 1, 300, 0, 12, 72, -1] ["his that's the were what would", "不敢不让心有选择有没有", "?", "could", "五万", "安排一周捕鲸船等着元月能早点期待", "", "很难说所以美好哦啊", "you time do there then", "come"] [] ["--", "季度", "?", "网管中心", "没时间你家探明哪方面生意重命名电话本"] ["good", "新心情", "that's but yeah did some get or did all would", "--", "he tell the didn't did why be could"] ["歉意", "-", "高级数码港欣慰好不好", "", "?", "who then hey up", "to so", "was you're"] ["-", "that's well he's yeah", "on"] ["排名意思专业稍等", "引进过来", "f", "say", "运行"] ["up one", "操作自信信息", "?"] ["e", "q", "q"] ["b", "n", "e", "z", "u"] [] [] [2024-07-01, 2023-12-14, 2025-02-17] [2024-02-18, 2023-12-17, 2024-02-18, 2024-02-18, 2023-12-13] [2025-02-18, 2026-02-18, 9999-12-31, 2024-01-08, 2025-02-18, 2025-02-18, 2027-01-09, 2023-12-17, 2023-12-12, 2024-02-18] [2025-06-18, 2023-12-18, 2014-08-12] [2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00] [2024-01-19 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00] [2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00, 2025-02-18 00:00:00] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00] [2023-12-12 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00] [2023-12-10 00:00:00, 2025-02-17 00:00:00, 2024-01-17 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2013-09-12 00:00:00, 2023-12-10 00:00:00] [2023-12-15 00:00:00, 2027-01-09 00:00:00, 2027-01-09 00:00:00] [2023-12-19 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123000, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] \N \N [2024-01-08 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00] +193 32679 1470356332 0 1604646174 2024-01-08 9999-12-31 2023-12-10 9999-12-31 带来 j 明辉大厦大雪制作大品牌累了比爱白高高 k y 嘉运达 p they you're on my see time \N 1 1.200000 0.000000 81.175700 40.123000 2024-02-18 00:00:00 2024-01-19 00:00:00 2023-12-11 00:00:00 2023-12-20 00:00:00 2027-01-09 00:00:00 2024-07-01 00:00:00 2024-01-08 00:00:00 2024-02-18 00:00:00 [0, 0, 0] [1, 1, 0] [1, 0, 0] [] [-128, 0, 0, 0, -1] \N [0, -128, 0, -128, 0] [] [-14822, 0, -1, -32768, 32767, -32768, 335, 23539, -15187, -317] [3488, -22716, 32767] [-19593, -1, 1, 24320, -19363] [-1, 0, -1, 27121, -16983] [] \N [-2147483648, 1810002400, 32679, -1290771028, 32679] [] [32679, -3884910751702926611, -1] [245, 245, 245] [-1, 9223372036854775807, 278491082, 147483648, -811174576, 6151347308629386492, -1, 147483648, 32679, -4792178365050365061] [-9223372036854775808, -1439661629959312963, 147483648, 6545221753992496805, -5286034254302407109, -7774579299553447657, 7817152259635060613, -9223372036854775808] \N [-1, 9, 0, 1, 0, 6, 387012452, 1, 9, 117834145] [3, -243850050, -2] [] [300.343000000000, 69.036600000000, 23.168400000000, 27.063900000000, 100.020000000000] [] [61.029100000000, -1.200000000000, 100.020000000000, -1.200000000000, 1.200000000000] [0.000000000000, 85.129100000000, 45.054900000000, 0.000000000000, 16.039700000000] [1.2000, 37.0519, -1.2000, 50.1828, 100.0200] [0.0000, -1.2000, 59.0061, 1.2000, 0.0000, -1.2000, 40.1230, 43.0985] [1.2000, 300.3430, 62.0846, 90.1698, 86.0099, 96.1550, 40.1230, 300.3430] [] [0, 82, 98] \N [300, -1, 0, 86, 49, 1, -1, 0] [300, 0, 40] [] ["作假关于后悔温总备案一句话赵总", "--", "from well okay is of we it", "", "启动", "n", "", "盘点", "-", "-"] [] [] ["ok", "喝得多最直接比爱白金成很大", "", "?", "can't her i can we I'll know up", "back I'm", "?", "没有"] ["back will now well look", "系统集成商动态焦作市暴风影音捕鲸船对自己质保金科贸", "?", "u", "之间", "?", "like tell no we", "--", "确实景峰知己大量现货爱国运费普庆乔东刘雪", "is can from don't will yeah would"] ["", "did it's or all some it's his", "-"] ["i now I'm", "e", "实业", "收吧", "like", "访问量停产短句", "", "售后", "--", "类似"] ["d", "x", "u", "w", "c", "c", "k", "w"] \N ["o", "i", "d", "a", "d", "c", "h", "g"] ["j", "l", "c", "k", "r", "c", "v", "e"] [2024-07-01, 2023-12-14, 2024-01-17, 2024-06-30, 2026-01-18, 2025-02-18, 2024-08-03, 2024-01-09] [2024-01-17, 2024-06-30, 2024-01-31] [2014-08-12, 2025-02-18, 2024-01-19] [2025-06-18, 2024-06-30, 2023-01-15] \N [2024-02-18 00:00:00, 2024-07-01 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00] [9999-12-31 23:59:59, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2024-08-03 13:08:30] [2003-08-27 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2023-12-12 00:00:00, 2027-01-09 00:00:00] [2023-12-15 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-14 00:00:00, 2014-08-12 00:00:00, 2023-12-19 00:00:00, 2024-01-17 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00] [2023-12-18 00:00:00, 2014-08-12 00:00:00, 2024-07-01 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00, 2023-12-13 00:00:00, 2008-01-22 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] [2023-12-16 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00] [2024-07-01 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123, 2027-01-09 00:00:00, 2026-02-18 00:00:00] [2015-10-03 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00] +194 -1 1193909353 2147483647 -2147483648 2023-01-15 2023-12-17 2023-12-14 2024-02-18 - 剪掉 \N -- because so of so 明天找我后者 ? 1 1 -1.200000 28.096200 1.200000 1.200000 2024-06-30 12:01:02.123000 2023-12-19 00:00:00 2024-06-30 12:01:02.123000 2023-12-20 00:00:00 2024-01-17 00:00:00 2027-01-09 00:00:00 2023-12-17 00:00:00 9999-12-31 00:00:00 [0, 0, 0, 1, 1] [1, 0, 1, 0, 0] [1, 0, 0, 0, 1, 0, 1, 1, 0, 1] [1, 0, 1, 0, 1, 0, 0, 1] \N [127, 59, -40, 5, -97, 36, 106, -128, -1, 56] [-128, 96, -1, 5, -122, 0, 1, -128] [-128, -1, 127] [140, 32767, -32768] [1, -32768, 11444, -32768, 32767, -22187, -32768, -32768, -4150, -1] [-17590, 27553, 32767] [245, -3806, 14015, -8665, 245, 1, 9399, 18662, 16325, 23163] [-821064761, 2147483647, 147483648, -1928369307, 2147483647] [-1, -1912763279, 147483648, 0, 32679, 1160124988, -154689134, -94547904, 0, 1] [] [-1191426314, 68443416, 526487666, 0, 32679] [6171307099690114426, 147483648, 0, 2711741286536369156, 147483648] [] [32679, -1360395183385122652, 9223372036854775807, -8360883886858654253, 8698416889881633720] [32679, 1776016, 5764438, -7559929847473574564, -1, 32679, -1, -9223372036854775808] [6, 0, 2, 505213021, 1, 155515402, -2, 0, -1, 8] [-187565537, 7, 1, 0, -2] [] [-777101914, -2, 8, 7, -1] [] [100.020000000000, 76.117000000000, -1.200000000000] [40.123000000000, 26.157600000000, -1.200000000000, 33.021000000000, 19.077300000000, 300.343000000000, 95.064000000000, 100.020000000000] [-1.200000000000, 14.081300000000, 1.200000000000, 1.200000000000, 300.343000000000, 0.000000000000, 300.343000000000, 1.200000000000, -1.200000000000, 96.149100000000] [40.1230, 0.0000, 0.0000, 14.0669, 40.1230, 0.0000, 40.1230, 54.1454, 100.0200, 1.2000] [1.2000, 300.3430, 1.2000, 36.1296, 100.0200] [100.0200, 28.0491, 300.3430] [14.0551, -1.2000, -1.2000, 0.1864, 300.3430, 1.2000, 40.1230, 300.3430] [91, 40, -1, 23, 300] [73, 81, 0] [-1, 1, 62, 300, 40, 73, 300, -1] [0, 300, 42, 0, 1] \N ["ok was hey it up do what him see and", "i", "不含税水平那天才好进账一部详细生病全名考前", "been", "日信黄黎明下了去掉吃饭哪词库刚刚这些话快车", "do would be at yeah now", "some", "h"] ["群英开着不能说愉快很错", "孙瑞霞", "say for his in for", "中华单子相约南阳路表情妈妈护", "--", "成绩", "?", "for have get ok would from"] ["强暴一切高级很难说", "副本", "it's he's I'll right", "just of so now", "?", "金牌经销总代理幕墙要不一款图腾月内这百年唱给", "关系", "tell"] ["that that's he's he like tell this my", "", "not", "查到前面多条", "mean"] ["--", "?", "I'll", "需求", "普庆", "--", "--", "for I'm do I'll had going is would the", "-", ""] ["千兆", "工程姑娘金鼎展开大会五月份内训不应该小苏工商局", "埋在", "with", "-"] ["算数心理学老大", "-", "he's now not there in", "多家", "海尔", "会话", "x", "机会", "see have think as", "c"] ["z", "v", "e", "j", "u", "h", "w", "c", "q", "a"] ["t", "n", "r", "j", "s"] [] ["j", "r", "o", "e", "i", "g", "n", "l", "w", "m"] [2024-02-18, 2023-01-15, 2023-12-13] [2025-02-18, 2024-01-09, 2014-08-12, 2023-12-10, 2023-12-10, 2024-01-08, 2014-08-12, 9999-12-31] [2023-12-17, 9999-12-31, 2024-01-31] [2023-12-13, 2025-02-18, 2023-12-12, 2024-01-19, 2013-01-11, 2023-12-17, 2025-06-18, 2023-12-19] [9999-12-31 23:59:59, 2024-01-17 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2004-11-05 00:00:00, 2026-01-18 00:00:00] [2023-12-19 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00] [2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2014-08-12 00:00:00, 2023-12-18 00:00:00] [2023-12-20 00:00:00, 2024-01-17 00:00:00, 2023-12-10 00:00:00] \N [] [9999-12-31 23:59:59, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00] [2026-01-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-31 00:00:00, 2024-01-19 00:00:00, 2023-12-10 00:00:00] \N [2023-12-11 00:00:00, 2027-01-09 00:00:00, 2014-08-12 00:00:00] [2023-01-15 08:32:59.123123, 2008-12-08 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59.123123, 2023-12-11 00:00:00] [2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00] +195 -2027440023 339381403 -1 -2147483648 2024-01-31 9999-12-31 2025-06-18 2024-01-17 you're 那里 -- right - so it that my so don't go the were right yeah could you're you who \N 1 1.200000 86.192400 1.200000 36.194400 2027-01-09 00:00:00 2024-01-17 00:00:00 2024-02-18 00:00:00 2023-12-19 00:00:00 2024-06-30 12:01:02.123000 \N 2027-01-16 00:00:00 2023-12-18 00:00:00 [0, 0, 0] [1, 0, 0] [0, 1, 0, 0, 1] [] [] [70, -99, 0, -1, -1] [7, -72, -114, 22, 5] [0, -25, -128, 3, -123] [-32768, 16413, -4710, 4578, -1, 32767, -838, 29048, 0, -25461] \N [5151, -1, 1, 23743, -1, -1, 11045, -1] [32767, 24237, 32767, 245, -31834] \N [] [-1, 1503084087, -1844102490, -1410681819, 0, 147483648, -2147483648, 0] [] [-9223372036854775808, 1, 2408102441760416036, -1, -8970852294082234339] [1599668254, 0, 9223372036854775807, -4665657244333079130, -1371156300] [5505213931900192924, -62376, 32679, 0, -4529647423094661020, 32679, -9223372036854775808, -841095624] [0, -1116190274, 245] [1319303529, 2070284995, 0, 1, -264564420] \N [0, -674411657, -1] [-2, 2, -2, -2, -62005241] \N [77.062700000000, 0.000000000000, -1.200000000000, 25.152000000000, 59.092100000000] [300.343000000000, 1.200000000000, -1.200000000000, 300.343000000000, 100.020000000000] [] [46.0168, -1.2000, 300.3430] \N [40.1230, 64.1972, 3.1183] [] \N \N [56, 1, -1, 16, 46, -1, 100, 0] [40, 40, 300, 69, 40, 1, 23, 40] [] ["", "几台后颈华你不工作吗批发邮政占有中旬", "天数太好了", "it's", "--"] [] ["", "tell have her", "", "--", "一些单子"] ["up", "能不能", "think", "意见", "地址", "昨天", "my as hey can", "something"] \N ["tell", "--", "when", "right get think", "可笑", "--", "--", "m", "mean", "-"] ["?", "实实在在", "yeah then would her will what he's for time ok", "昨天到达我刚回来呀开会下雪", "联通留一下去爱", "up out", "入职", "--", "谈谈", "me can't were yeah"] \N [] [] ["m", "d", "d", "t", "n"] [9999-12-31, 2024-06-30, 2023-12-14, 2023-12-15, 2024-02-18] [9999-12-31, 2014-08-12, 2025-06-18] [] [] [2014-08-12 00:00:00, 2023-12-19 00:00:00, 2024-01-08 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00] [2027-01-16 00:00:00, 2024-06-30 12:01:02, 2023-12-12 00:00:00, 2012-05-22 00:00:00, 9999-12-31 00:00:00, 2023-12-13 00:00:00, 2023-12-13 00:00:00, 2023-12-16 00:00:00] [2027-01-16 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00] [2023-12-19 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00] [2024-07-01 00:00:00, 2023-12-15 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2005-10-10 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2023-12-15 00:00:00, 2023-12-14 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2023-12-11 00:00:00, 2006-04-20 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-14 00:00:00] [2011-07-06 00:00:00, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 2024-01-08 00:00:00, 2025-02-18 00:00:00] [9999-12-31 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00] [] [2023-12-19 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00] [2026-01-18 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00] +196 61496289 -287587316 147483648 2147483647 2023-12-13 2024-01-09 2023-12-18 2025-06-18 焦作市小键盘 oh 不传东海家里水哦莱克特会话支持高一些聊天 \N here tell why now be see we mean ? 排名彩页中原 i 1 1 12.077000 100.020000 300.343000 1.200000 2025-06-18 00:00:00 \N 2023-12-19 00:00:00 2023-12-19 00:00:00 2024-07-01 00:00:00 2024-01-08 00:00:00 2023-12-11 00:00:00 2025-06-18 00:00:00 [1, 1, 1, 0, 0, 0, 1, 0, 0, 1] [1, 1, 0, 0, 0, 1, 1, 1, 1, 0] [0, 0, 0, 0, 1] [1, 1, 1, 0, 1, 1, 1, 0] [-128, 127, -128] [] [-52, 127, -128, -128, 1, 126, -1, -30, 0, 1] [0, 127, 1, 27, -128] [14969, -25584, 28454, -16936, -19482, -32768, 11261, 0, 4099, 245] [29719, 245, 23818, -32768, 18151, 0, -7239, 8332] [1, 10779, 23795, -12020, -1] [-17521, 24018, -32768, -4665, 8193] [-2083057250, -1630978732, 2147483647, -1, -1959532085, -1, -763088881, -2147483648] [-2032295658, -2147483648, -465726957, 1, 782115750, 0, 1, -891839724, 147483648, 2147483647] [2147483647, 32679, -2147483648, -712445928, 147483648, -2147483648, 0, -65616973, -2147483648, 2147483647] [-2147483648, 2090165156, -2147483648, 1633164420, -2147483648] [147483648, 9223372036854775807, 1, 9223372036854775807, 32679] [] [1, 1, -1372015950, 32679, -9223372036854775808, 147483648, -2763761682781680522, 9223372036854775807, 147483648, -358214308] [] \N [-1, 1091919237, 1660908756, 372389676, 1, 117132396, 0, 3, 1, 1] [] [0, 1, 3, 1, 1, 1, 1, -2] [68.167300000000, 100.020000000000, 40.123000000000, 100.020000000000, 46.021100000000, 100.020000000000, 300.343000000000, 300.343000000000] [40.123000000000, 100.020000000000, 0.000000000000, 100.020000000000, 100.020000000000, 300.343000000000, 100.020000000000, 97.121300000000] [0.000000000000, 0.000000000000, 0.000000000000, 82.075700000000, 100.020000000000, 98.038200000000, 0.000000000000, 0.000000000000, 1.200000000000, 100.020000000000] [100.020000000000, 66.022800000000, 40.086900000000, 40.123000000000, 4.062200000000] [40.1230, 0.0000, 100.0200, 40.1230, 69.0440] \N [32.1871, 0.0000, 74.1217] [1.2000, 100.0200, 1.2000, 40.1230, 0.0000] [90, 17, 21, 1, 100] [22, 38, 300, 300, 64] [0, 57, 300, 40, 1, 100, 100, 11] [300, 59, 96] ["tell", "h", "b", "she", "", "she", "okay", "me"] ["o", "农历", ""] ["小故事第一分别五号疑问全程", "国标它是疑问阿奎那孟磊麻烦死武汉", "郑娜做事", "", "?", "it's", "have hey up not", "-", "拿下", "j"] ["because out see", "do", "you're", "-", "?", "新版", "then", "-", "-", "客运量明珠智慧重点我在政治哦玉冰凌峰应该侠诺洗衣服"] ["屏保纯平邮箱十来年", "-", "?", "d", "--", "-", "from", "been a some you're me how see do for"] ["get your like were tell who you this hey", "送过去放心中庸说不定指纹高科补上那一片党经理华强", "one he ok that him it", "do ok would good hey time no got", "didn't", "what", "交货这些张经理", "what", "盈盈收集好好新乡市已给水哦知道了不走", "up had be"] [] ["对面日益数码你好吗景霞生活大河压力容量", "-", "产品事业部经理", "one been as but this were that's look oh say", "a"] ["b", "j", "k", "e", "v", "x", "c", "n"] ["k", "u", "m", "z", "q", "t", "e", "x", "r", "v"] [] ["f", "o", "b", "b", "p", "d", "u", "a", "p", "b"] [2024-02-18, 2023-12-20, 2023-12-14, 2025-06-18, 2023-12-18] \N [2025-02-17, 2023-12-14, 2023-12-16, 2023-12-10, 2024-07-01, 2024-01-19, 2012-10-14, 2023-12-14, 2025-02-18, 2024-02-18] [2024-02-18, 2026-02-18, 2023-12-19, 2018-11-15, 2026-02-18] [2024-01-31 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2025-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 9999-12-31 23:59:59, 2024-02-18 00:00:00] [2024-01-08 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00] [2023-12-11 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2003-02-10 00:00:00, 2024-01-08 00:00:00] [2023-12-20 00:00:00, 2023-01-15 08:32:59.123000, 2024-06-30 12:01:02.123000, 2023-12-16 00:00:00, 2024-08-03 13:08:30] [2023-12-11 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00] [] [2025-02-18 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 2023-12-20 00:00:00] [2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2011-05-05 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00] [2026-02-18 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-18 00:00:00] +197 0 -1087168164 -1451438104 162365621 2027-01-16 2024-01-31 9999-12-31 2023-12-09 there for going h b ? i no - 飞信底价磁带机 \N 1 1.200000 40.123000 1.200000 33.032000 2023-12-15 00:00:00 2019-10-13 00:00:00 2023-12-14 00:00:00 2023-12-12 00:00:00 2023-12-20 00:00:00 2024-07-01 00:00:00 2023-12-16 00:00:00 2024-02-18 00:00:00 [1, 0, 0, 1, 1] \N [0, 1, 0, 0, 1] [0, 1, 0, 1, 1, 1, 0, 1] [] [] [] [127, 1, 127, -128, -1] \N [3375, -13813, -15609, -24030, 0, -1, -4721, -1551] [4858, -5224, -32768, -3999, 7519, 1, 8356, 1, 3951, -32768] [26275, -32768, 245, 245, 1, -21812, -30645, 20250, 32767, 1] [-1, -29614184, 2089734521] [611725586, -2147483648, 0, -1306242501, 1189492593] [1832768645, 32679, -2091388925] [-1329290632, 32679, 2147483647, 543381457, -260186463] [-9223372036854775808, 1, 147483648] [] [-8743952911148653547, 0, 6645731046458472507, -1, -8214268897064243424, -6254809, 6866109729612310062, 1] [0, -9223372036854775808, 245, 0, 4052487] [] \N [] [-2, 7, -1, 0, -2, 533073877, -2, -1, 0, 0] [] \N [-1.200000000000, 0.000000000000, 300.343000000000, 40.123000000000, 11.151800000000] [] \N [57.0015, 57.1636, 1.2000, 2.1832, 300.3430] [300.3430, 300.3430, 69.1152] [0.0000, 0.0000, 0.0000, 0.0000, 0.0000, -1.2000, -1.2000, 100.0200] [0, 0, 15, 0, 0] [56, 39, 82, 0, 1] [17, 0, 300, -1, 8, 40, 40, 27, 0, 0] [] ["-", "没有到货", "令你上门多少钱琪英天华请客拆机看法身份", "had", "or there time"] ["now him", "", "实际上", "want", "否则背书共享资格迪微走之前稍微生物"] ["v", "so", "was", "华科", "do he just look her not yeah good I'll", "l", "短信初次有种不太确定北京发票天晴红雷眼泪", "小王"] ["--", "天下无贼", "right", "--", "and", "", "邮寄一下阿远大一段这部分跟进中多家熊朝华牌子重复", "屠龙记"] \N ["不见隐隐约约壹佰所谓这个扩充高压贰台神偷", "were are been a look oh", "-", "-", ""] [] [] ["f", "n", "k", "o", "z", "f", "k", "e", "o", "y"] ["w", "e", "s", "n", "o", "w", "w", "t", "d", "z"] ["y", "x", "j"] ["l", "r", "k", "v", "y", "e", "q", "p"] [2023-12-12, 2024-08-03, 2024-01-08, 2006-11-18, 2024-02-18] \N [] [] [2024-06-30 12:01:02, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2024-01-31 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00] [2023-12-11 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00] [2024-01-09 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 2024-01-19 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2024-01-08 00:00:00, 2023-12-12 00:00:00] \N [2023-12-12 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00] [2023-12-16 00:00:00, 2023-12-19 00:00:00, 2023-12-11 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00] [9999-12-31 23:59:59, 2027-01-09 00:00:00, 2023-12-09 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00] [2024-01-17 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59] [2026-02-18 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123123, 2024-01-09 00:00:00] [2023-12-13 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59.123123] [] +198 \N 0 -1792963647 -1845477502 2023-01-15 2023-12-11 2025-02-18 2026-02-18 泰杰斯 以前主办红颜百家你这个这段 这样浮动组哟大公司动画拿下服务他们台式机 淀雪核心月份只要不太确定无限 总业绩网络 ? - 暂时黄经理发过张婧俐考验不让人员名单 0 0 16.075900 83.122200 1.200000 -1.200000 \N 2014-08-12 00:00:00 2014-08-12 00:00:00 2025-06-18 00:00:00 9999-12-31 23:59:59 2023-01-15 08:32:59.123123 2024-01-19 00:00:00 2023-12-12 00:00:00 [0, 1, 1, 1, 1, 1, 0, 0] [0, 1, 0] [] [1, 0, 1] [0, 14, 3] [113, 66, -128, 57, -61, 1, 0, 1] [11, 127, 3, 0, 120] [2, 1, 127, -1, -58, -34, -8, -31] [] [-18357, 32767, -32768, 0, -1, 0, 4377, -7347] [] [-31249, 245, 245, 1039, -6783] [-1150910293, 673750355, 1056360692, -1, -1318294508, 1522004783, 1173560463, -1, 32679, -1631074101] \N [1, -750378701, 1, -1, -1, 0, 39454696, -1929823501] [1, -1591036720, -1279764855, 1982279435, -395428402, 778597860, 0, 147483648, -2097811496, -1066062394] \N [-171805351, -2592902450535362506, -72082039115742936, -4209260531267608572, -8396612732375936263, 8232648, 3106937, -9223372036854775808] [-1418713026, 1, 32679, -1, 8695890104697209393, 0, 9223372036854775807, -7569158] [] [4, -2, -2, -2, 7] [-2, -1, 0, 1, -1946424618, -1548811761, 1843402063, 5, -945679890, 481859936] [] [1, 1884737936, -1, 1, -2, 2, -2, -1] [100.020000000000, 0.000000000000, 46.075700000000, 100.020000000000, 84.199500000000, 40.123000000000, 100.020000000000, 40.123000000000, 0.000000000000, 300.343000000000] [] [39.034800000000, 1.200000000000, 0.000000000000, 40.123000000000, 300.343000000000] [1.200000000000, 0.000000000000, 300.343000000000] [100.0200, 98.0495, 300.3430] [300.3430, 89.1242, 100.0200, -1.2000, 39.1830] [1.2000, 0.0000, 71.1375, 100.0200, 85.0633, 300.3430, 1.2000, 100.0200, 54.1144, -1.2000] [1.2000, 99.0740, 0.0000, 1.2000, 100.0200] \N [78, 37, 76, 30, 47, -1, 72, -1, 57, 0] [40, 100, 0] [-1, 40, 300, 27, 300] ["my", "", "well know this you there how want well", "", "--", "the", "to look we as didn't are hey have", "经理", "刀片鉴别特性那段一年中一种拿下宏泰那天", "y"] ["out", "could", "网易", "", "--", "--", "his", "him ok why i right I'm know know right", "have I'm in come I'll go", "get would the but here"] [] ["--", "谦虚", "所要求", "-", "等着", "--", "here", "I'm"] ["中心广益", "look", "did with it now at right he got go", "can't", "同学招待费叫做转移长光", "o", "?", ""] ["?", "-", "who how", "-", "尝试", "", "天威", ""] ["w", "how up right we or is", "one about yeah well out", "some say of mean he that's", "-", "", "--", "岩春公司该国长了不曾意义正式日常", "I'm right or come with hey not oh well then", "time"] [] ["n", "i", "w", "l", "j"] ["l", "o", "b", "h", "k", "r", "m", "i"] ["u", "j", "q"] [] [2024-01-09, 2024-01-09, 2024-02-18, 2025-06-18, 2023-12-09, 2023-12-14, 2025-06-18, 9999-12-31, 2024-01-19, 2023-12-09] [2025-06-18, 2024-06-30, 2023-12-10, 2023-12-14, 2024-02-18, 2024-06-30, 2024-02-18, 2023-12-09, 2025-06-18, 2026-02-18] [2023-12-14, 2023-12-17, 2023-12-11, 2024-02-18, 2026-01-18, 2018-10-03, 2023-12-13, 2025-02-18, 2024-02-18, 2024-06-30] [2024-01-17, 2023-12-13, 2024-01-31] [2025-02-18 00:00:00, 2003-04-03 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00] [2024-01-08 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59, 2024-01-31 00:00:00, 2024-01-31 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00] [] [2023-12-17 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00] [2023-01-15 08:32:59.123000, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-17 00:00:00] [2023-12-20 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00] [] \N \N [2023-12-12 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2009-01-08 00:00:00, 2024-07-01 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00] [] +199 \N 2147483647 -2147483648 32679 2025-02-18 9999-12-31 2027-01-16 2024-02-18 -- -- - n z were ok 回答问题 \N 1 \N 40.123000 300.343000 19.114000 2005-07-20 00:00:00 2025-02-17 00:00:00 2024-01-09 00:00:00 2024-08-03 13:08:30 \N 2006-03-20 00:00:00 2005-08-25 00:00:00 2024-01-09 00:00:00 [1, 1, 1, 1, 0, 1, 1, 1, 0, 1] [1, 1, 0] [0, 0, 0, 1, 1] [1, 0, 1, 0, 1, 1, 1, 0] [60, 80, -11, -128, 1, 1, -46, 1] [-82, -128, 0] [] [80, 8, 0, -114, 0, 1, -1, 98, 127, 18] \N [0, 21182, 32767] [-29842, 0, 30416, 245, 27148, -8223, 24528, 0, 32767, -21786] [-15081, 1, -18418, 0, 32767, 20199, 0, -1, -1, 30696] [2147483647, 0, -2147483648, -15658762, 790547913] \N [-1023824052, 0, 0, -1, -1] [649013265, -1, -1603338831] \N [4994969398251608309, 0, 6242572617506329952, -1, 147483648, -9223372036854775808, 0, 989252106788785567, -9223372036854775808, 5729299446201365628] [1068244, -9223372036854775808, 0, 0, 178351646903725695, 9223372036854775807, -388501072, -1146897551042666770, -5298928908669271617, -6050365106633715534] [147483648, 32679, 3585230, 3841091479946097126, 9223372036854775807, 32679, -9223372036854775808, 32679] [] [1, 0, 4, 6, 0, 1, -2, 463443862, -2, 1] [-1, -1, 0] [] \N [69.137300000000, 0.000000000000, 1.200000000000, 100.020000000000, 100.020000000000] [] [0.000000000000, 100.020000000000, 5.054800000000, -1.200000000000, 21.008400000000] [300.3430, -1.2000, 0.0000] [-1.2000, 100.0200, -1.2000, 40.1230, 300.3430] [] [100.0200, 40.1230, 0.0000, 100.0200, 0.0000, -1.2000, 14.1214, 100.0200, 47.0278, 4.0080] [40, 79, 97, 77, 1, 40, 1, 300, 0, 40] [1, 100, -1, 0, 46, -1, 40, 300, 100, -1] [] [40, 90, 42, 40, 40, 40, 22, 100, 1, -1] ["出现", "贵姓", "--", "平均不够对吧其中欧", "?", "磁盘", "繁体字", "", "go for get and", "调整"] ["yeah oh want is one look here about or can't", "hey got is were that's time do could", "?", "大雪今年", "?", "they", "tell I'll on he's", "-", "", "进入了毕业墙南三环"] [] ["透露经过青春刘亮博源", "going", "for one I'll her he hey there"] ["--", "吹去多条开发打电话授权经销总代理侠诺环保性公司都说常用", "--"] ["?", "--", "?", "", "遮盖多好申请谁那里", "-", "看你月亮世纪图标接受旅游尊敬拿住", "can't that going can't on who on how here", "一开始时间少黄河科学四核我给你打吧果断说吧问起你还不去", "?"] [] ["--", "her look ok was", "v", "虚妄公寓领取普通组装认证安阳恢复山特过程", "do", "r", "v", "go this because when can't don't just there", "签单", "钻石"] ["t", "o", "s", "s", "p", "k", "w", "a", "g", "q"] ["p", "v", "f", "m", "p"] ["m", "u", "k", "y", "y", "i", "p", "l", "z", "b"] ["c", "u", "i"] [] [2023-12-12, 2025-02-17, 2024-01-08, 2024-06-30, 2023-12-13, 2024-01-09, 2024-06-30, 2014-08-12, 2023-12-17, 2014-08-12] [2026-02-18, 2024-06-30, 2009-11-27, 2023-12-09, 9999-12-31, 2014-08-12, 2023-12-16, 2024-01-19, 2024-01-17, 2024-01-19] [2025-02-17, 2023-12-15, 2024-02-18, 2027-01-16, 2024-07-01, 2024-02-18, 2024-06-30, 2026-02-18] \N [9999-12-31 23:59:59, 2026-01-18 00:00:00, 2023-12-16 00:00:00, 2001-08-21 00:00:00, 2024-06-30 12:01:02] [2024-07-01 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00] [2023-12-17 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2005-04-07 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02, 2024-01-19 00:00:00] [2023-12-12 00:00:00, 2023-12-20 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2014-11-02 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00] [2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00] [] [9999-12-31 23:59:59, 2023-12-13 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-18 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00] [2023-12-09 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00, 2023-12-11 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00] [] [2024-01-09 00:00:00, 2023-12-13 00:00:00, 2023-01-15 08:32:59.123123, 2025-06-18 00:00:00, 2024-08-03 13:08:30] [] +200 -1 \N 152372286 0 2023-12-19 2024-07-01 2023-01-15 2026-02-18 喝口 人体油画 神秘 - like with you when we your his can't like that's think back will do something his for just been 联系人哪方面防治环保局目标过了南阳金总管理接收 0 1 76.173000 40.123000 40.123000 26.050900 2024-06-30 12:01:02.123000 9999-12-31 00:00:00 2025-06-18 00:00:00 2023-12-12 00:00:00 2024-01-17 00:00:00 2024-01-19 00:00:00 2023-12-19 00:00:00 2024-02-18 00:00:00 [0, 1, 0] [1, 1, 1, 0, 1] [0, 1, 1, 1, 1, 1, 1, 1, 1, 1] [] [8, 3, 106] \N [44, 21, 0, 127, 48] [9, 1, 1, -128, 0, 58, 0, 1, -128, -1] [16678, 10438, 1, -19805, 12326, 27622, 4840, -28338] [8536, -31947, -318, -32617, -24546, -9424, -11249, 25517] [1, 15821, 17718] [-6655, 11002, 245] [1012649467, 2147483647, 2107839310] [-561646402, 1, -749386843] [] [0, 1, -1] [32679, 0, 4063423785373582477] [32679, 0, -556404164883421909] [] [-9223372036854775808, 32679, -2858041, -931641769, 0, -9223372036854775808, 141729027, 32679] [0, 658787558, -2, 1, 3, -2, 0, -2] \N [-2, 1, -2, -1650713543, 1, 1, 0, -1] [] [0.000000000000, 5.056000000000, 74.040900000000, 0.000000000000, 70.165500000000, 0.000000000000, -1.200000000000, 40.123000000000, 31.003100000000, 100.020000000000] [23.003600000000, 1.200000000000, 100.020000000000] [100.020000000000, 58.188400000000, 33.160300000000, 100.020000000000, -1.200000000000, 1.200000000000, 0.000000000000, -1.200000000000] [87.081500000000, 1.200000000000, 1.200000000000] [0.0000, 78.1469, 81.0661, 40.1230, 85.0787, 40.1230, 1.2000, 40.1230, 13.0585, 4.1641] \N [34.0253, 0.0000, 6.0722] [40.1230, 83.1125, 0.0000] [] [71, 0, 100, 1, 23, 1, 42, 1] [89, 88, 61, 1, 0] [-1, 100, 300] [] ["蓝牙", "", "", "with on oh he out up for this him", "余下优先大姐田辉国际"] ["?", "mean", ""] [] ["okay", "-", "", "随其自然", "--", "走过去琵琶语启恒", "--", "about", "this", "so"] ["?", "偷出", "道理考试科技大厦订票不问停机", "not", "", "come", "-", "he", "打架当然爱具体台式机", "空下愿望哪方面"] ["一脚收条证书普通词语回答问题诺顿词条", "看你", "-", "头痛代理商小苏数哟把握流明会话", "got", "-", "don't good like are want if look now", "--", "on not he's as want something oh that's all no", "天天感受这点徐州留言加密只能代表"] [] ["x", "j", "g", "r", "y", "d", "n", "r"] ["g", "u", "y", "w", "x", "f", "x", "z", "s", "p"] [] ["k", "i", "y"] [9999-12-31, 2023-12-17, 2023-12-14] [2023-12-13, 2024-01-19, 2026-02-18, 2026-01-18, 2026-02-18, 2023-12-11, 2023-12-14, 2024-02-18] [2026-01-18, 2024-01-17, 2025-06-18, 2023-12-20, 2026-01-18, 2023-12-15, 2023-12-20, 2024-06-30, 2023-01-15, 2026-01-18] [2023-12-14, 2024-02-18, 2024-01-19, 2023-12-16, 2023-12-16, 2023-01-15, 2023-12-18, 2023-12-20] [9999-12-31 23:59:59, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59] [2023-12-13 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00] [2023-12-13 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02, 2023-12-15 00:00:00, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2027-01-09 00:00:00] [2023-12-14 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00] [2024-06-30 12:01:02.123000, 2024-07-01 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2009-05-15 00:00:00, 2024-01-08 00:00:00] [] [9999-12-31 00:00:00, 2026-01-18 00:00:00, 2011-11-15 00:00:00, 2024-01-19 00:00:00, 2023-12-11 00:00:00] [2023-12-10 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00] [2025-02-18 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2023-12-17 00:00:00, 2004-12-12 00:00:00, 2023-12-12 00:00:00, 2023-01-15 08:32:59.123123] [2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-16 00:00:00, 2023-12-16 00:00:00] [2023-12-17 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2023-12-11 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00] [2014-08-12 00:00:00, 2023-12-15 00:00:00, 2023-12-13 00:00:00] +201 1996348823 -4959115 0 -2129882235 2025-02-18 2007-03-14 2027-01-16 2026-01-18 that's could in were it of think when not 达成率 - -- q then can do can't think 0 1 0.000000 40.123000 1.200000 300.343000 2026-01-18 00:00:00 2024-01-19 00:00:00 2023-12-16 00:00:00 9999-12-31 00:00:00 2025-02-18 00:00:00 2027-01-16 00:00:00 2024-07-01 00:00:00 2024-01-19 00:00:00 [0, 0, 1, 1, 0, 0, 1, 1, 1, 0] [0, 1, 1, 0, 0, 1, 0, 1, 0, 0] [0, 1, 0, 0, 0] [1, 1, 0, 1, 0] [98, 43, 0, 127, -128, -51, -1, 119, -9, -103] [] [-33, 1, 0, 0, 1] [2, 1, -1] [29027, -1, -13140, -1, 245] [] [11402, 28364, 32405, 245, 1, 0, 15605, 32767, -32194, 1] [1, -1, -19318, -7989, -1] [-2147483648, 1290303719, 250117944, -1564684730, 147483648, -2114749973, 147483648, -2147483648, -379643315, 1516420302] [] [0, 0, -1] [] [0, 7987042, 1, 9223372036854775807, 4209466846328003752, -9223372036854775808, 147483648, 1] [32679, 0, -7815166432448022309, 8187387940759030312, 1529293902] [0, -9223372036854775808, -1] [] \N \N [] [] \N [1.200000000000, 18.110200000000, 92.180700000000, 81.088000000000, 64.006800000000, 42.092700000000, 0.000000000000, 1.200000000000, 42.001200000000, 1.200000000000] [100.020000000000, -1.200000000000, 100.020000000000, 40.123000000000, 100.020000000000] [6.145000000000, 40.123000000000, 7.013000000000, 100.020000000000, 40.123000000000, -1.200000000000, 72.039800000000, 50.168600000000] [50.1076, -1.2000, 0.0000, 1.2000, 300.3430, 70.0615, 63.0294, 78.0809] [100.0200, 300.3430, 50.0458, 0.0000, 100.0200, 42.0836, 300.3430, 1.2000, 1.2000, 1.2000] [2.1799, 300.3430, 63.0221] [100.0200, 300.3430, -1.2000, 300.3430, 1.2000, 22.1439, 300.3430, 100.0200, 0.0000, 33.0638] [-1, 0, 1, 100, 0, 0, 300, 40, 1, 40] [83, 40, 1, 100, 80] [1, 37, 1, 1, 100] [11, 50, 40, 1, 1, 40, 40, 38, -1, 300] ["t", "-", "--", "瑞和先锋瑞科前两天信任", "go the something have right we get is this but", "-", "w", "what", "okay", "is"] ["中层经理", "w", "-", "", "just", "去享受设和彭伟都说不信立博预防交局", "郑娜", "what go up in look was been up"] ["没错", "--", "do", "", "been don't i he's as was not get"] ["--", "-", "o", "--", "-", "you're did why mean see that's ok he's did why", "a about that are you all how you're", "he", "据对", "?"] ["one didn't were what his", "have when", "-"] ["要是", "道歉", "之后一天", "?", "-", "all can't he's tell well this right", "r", "?"] [] ["be will his", "he's", "when going up can this something", "", "这么做小庞冲淡周保全儿童迪微到货私人"] ["r", "o", "s"] ["s", "b", "f", "u", "v", "x", "i", "g", "z", "o"] [] ["i", "r", "k", "h", "f", "r", "e", "o"] [2013-03-23, 2024-02-18, 2024-08-03, 2024-01-17, 2024-01-09] [2024-07-01, 2023-12-14, 2024-01-08] [2024-02-18, 2012-02-13, 2027-01-16, 2024-02-18, 2023-12-14, 2024-01-17, 2023-12-12, 2025-02-17] [] [2023-12-20 00:00:00, 2023-12-15 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2023-12-14 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2023-12-09 00:00:00, 2024-07-01 00:00:00] [2024-01-09 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59] \N [] [2023-12-15 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2014-08-12 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00] [2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2016-02-14 00:00:00] [2023-12-20 00:00:00, 2023-12-09 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00] [2024-07-01 00:00:00, 2026-01-18 00:00:00, 2023-12-19 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00] [2025-02-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00] [2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00] +202 32679 32679 863588177 1 2023-12-11 2027-01-09 2025-02-17 2014-08-12 something for there see just 小浪底信号还向等会残品配置参数蠢材稍等计划办好 交通摄像头 一共元月很难研修网中天金牌站在文华 at then do can't look going didn't about oh right okay look 我们见不了 0 1 \N 1.200000 0.000000 40.123000 2010-09-09 00:00:00 2024-08-03 13:08:30 2023-12-09 00:00:00 2025-02-18 00:00:00 2024-02-18 00:00:00 2024-01-31 00:00:00 9999-12-31 23:59:59 2027-01-09 00:00:00 [1, 1, 1, 0, 1] [0, 1, 1, 0, 0, 0, 1, 0] [0, 0, 1, 1, 1] [0, 1, 1, 0, 1, 0, 0, 0] [-71, 127, 0, 127, -1] [103, -35, -128] [-128, 6, -32] [6, 127, -100, -1, -19] [32767, 27764, 26272] [-32000, 245, 245, 245, 32767] [-2311, 27706, 13072] [] [1, -192772849, 2147483647] [0, 2147483647, 147483648, 32679, 1239966243, 494831272, -1050453841, -296811309, 533777987, 147483648] [1606443502, -1, -934553138] [-1022510688, -2147483648, -82096202, 0, 147483648, 1, 147483648, -178808114, 1, 1051851178] [0, 0, 245, 147483648, 1579661549142272724, 5232542, 147483648, 161352874] [4017298910009548701, 0, -1, 147483648, -9039445761040425625, -808767526, 3282833, 3422093694348823514, 147483648, 245] [245, 620465882, 6359163, -1, -1, 245, -1, 1] [-1561640, 147483648, 1159025325, 1, 147483648, 1, -1, 147483648, 32679, -4798771067002479147] [-1, 1418319380, 7, 0, -2, -1, -2, 1622838528, 0, -1] [6, 0, 1, -2, -1000225168] [-1, 8, -1, -2, 0] [] [1.200000000000, 77.131000000000, 1.200000000000, 82.141300000000, 100.020000000000, 300.343000000000, 75.157800000000, 3.179200000000] [56.021700000000, 31.059700000000, -1.200000000000, 0.000000000000, 0.000000000000, 100.020000000000, 23.061100000000, 40.123000000000, 74.102400000000, -1.200000000000] [] [1.200000000000, 0.000000000000, 92.166300000000, 0.000000000000, 100.020000000000, -1.200000000000, 16.131900000000, 300.343000000000, 69.169900000000, 87.167900000000] [90.1161, 0.0000, 40.1230, 42.0891, 71.1091, 54.1915, 1.2000, 40.1230] [0.0000, 42.1054, 1.2000] [0.0000, 1.2000, 12.1936, 23.1535, 40.1230] [97.0300, 36.0140, 1.2000] [300, 300, -1] [100, 89, 300] [] [] ["some something know", "--", "网上科技其中欧我不软件园小王七匹狼", "回复", "", "推出", "零售", "宝宝", "-", "so come or a they didn't could get"] ["mean a see", "-", "n", "k", "?", "as about ok had you for one and from", "m", "?", "?", "j"] ["金科华", "can didn't now my good him now did me he", "there want have for he's it there", "--", "-", "了解", "n", "why one do think could", "客服", "视讯"] ["look", "中间轻轻吃的及其上门图美同名研究院双子来说", "up", "伏牛路", "just"] ["?", "mean", "", "-", "he"] ["?", "we", "?"] ["-", "--", "d", "have", "-", "-", "would for there I'm as who right why you is", "晓燕"] ["?", "-", "历来", "go I'm at his you're because", "a"] [] \N ["h", "q", "n", "g", "r", "f", "p", "c"] ["l", "e", "j", "i", "u", "h", "v", "d", "v", "n"] [2025-06-18, 2014-08-12, 2023-12-17, 2024-07-01, 2023-12-10, 2023-12-10, 2024-01-31, 2023-12-17, 2014-08-12, 2025-02-17] [2024-06-30, 2024-02-18, 2014-08-12, 2024-01-08, 2023-12-11, 2025-06-18, 2024-01-09, 2024-06-30, 2024-08-03, 2023-01-15] [2025-06-18, 2023-12-12, 2024-01-19, 9999-12-31, 2023-12-18, 2026-02-18, 2024-02-18, 2024-01-17, 2024-02-18, 2023-12-15] [2023-12-17, 2023-12-09, 9999-12-31, 2023-12-12, 2023-12-16] [9999-12-31 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2005-01-15 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2024-01-17 00:00:00, 2002-03-18 00:00:00] [2023-12-12 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2002-06-09 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00] [2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-19 00:00:00, 2023-12-11 00:00:00, 2023-12-15 00:00:00, 2009-02-26 00:00:00, 2023-12-19 00:00:00, 2024-07-01 00:00:00] [2026-01-18 00:00:00, 2024-06-30 12:01:02, 2024-07-01 00:00:00, 2026-01-18 00:00:00, 2023-12-16 00:00:00] [] [] [2014-08-12 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-12 00:00:00, 2023-12-17 00:00:00, 2011-07-18 00:00:00, 2025-02-17 00:00:00, 2024-01-31 00:00:00] [2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00] [2023-12-18 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00] [2024-01-19 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2010-05-03 00:00:00, 2024-01-17 00:00:00] [2023-12-17 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [] +203 -988870900 0 638971526 1501008411 2024-01-31 2027-01-09 9999-12-31 2024-02-18 西开发 u oh - 另存为停产很好我一会给大精英们中铁娃娃驱动广通你杂 ? 金蝶对咱们查查齐村关系太多国奥广角沃尔普 质量 \N 1 300.343000 46.067600 -1.200000 11.013600 9999-12-31 23:59:59 2024-01-19 00:00:00 2023-12-11 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2023-12-09 00:00:00 2023-12-10 00:00:00 [1, 1, 0, 0, 1] \N [1, 1, 1, 1, 0, 1, 1, 1] [1, 0, 1] [1, 0, 5, 2, 49] [31, 127, 7, 127, -112, 36, 0, -128] [0, 70, 0] [] \N [274, 30617, 590, 0, -32768, 1, 245, -6915, 24736, 0] [-28928, 27870, 1, 0, 1] [-32768, 13650, 4581, 32767, -1492, 84, -13681, 13585] [739417994, -1112212146, 0] [32679, -2147483648, 1, 1, 1] [-254387696, 1581913097, 812448938] [1827810795, -1524164236, -803976996] [32679, 0, 5333306209275552341, 245, 6941251, -4444908, -9223372036854775808, 1] [-1, 0, 9223372036854775807, 9223372036854775807, 30854569, 0, -2136667734726733937, 245, 1, -7880542] [1532117077, -9223372036854775808, 9223372036854775807, 9223372036854775807, 0, -103829, -1, 147483648] [-9223372036854775808, 1, -9112933757426227518, -1441980975392217072, 0] [1, 1822493668, 1, -1, -1] [-1, -2, -1, -2, 0, 0, 1, 1, -2, 0] [-2, -1830933534, -2, 1, 0] [] [31.182300000000, 31.016000000000, 38.024600000000, 100.020000000000, -1.200000000000, 1.200000000000, 86.176700000000, 89.179300000000] \N [300.343000000000, 19.037500000000, -1.200000000000, 100.020000000000, 100.020000000000] [-1.200000000000, 93.187500000000, 0.000000000000, 91.191700000000, 25.188700000000, 8.181300000000, 32.018700000000, 40.123000000000] \N [-1.2000, 40.1230, 40.1230, -1.2000, 40.1230, 1.2000, 100.0200, 32.0540, 40.1230, 55.0443] [0.0000, 75.1211, 40.1230, 39.1558, 0.0000, 1.2000, 7.1802, 0.0000, 0.0000, 92.1439] [40.1230, 100.0200, 300.3430] [29, -1, 100] [0, 300, 300, -1, 100, 300, 58, 300, 1, 40] [67, 19, 1] [60, -1, 46, 1, 81, 300, 10, -1, 68, 100] ["伤心夏娜吃肉金信离谱弟子才到不起", "something be", "w", "培训含有铿锵冀海潮礼数", "?", "深入", "v", "刻骨铭心", "like what think I'll I'll her", "-"] ["here", "青青草人选下半年费用", "what out on her the good had hey", "", "还好吧", "以前动态博源划分", "e", "that out did here think did well what want"] [] ["is his we could didn't can i be now one", "人名核对过双机", "-", "?", "回来新买多少钱周报组装会话轩明吃过饭完成率", "比人", "--", "商家欧洲商务新亚非", "六万", "--"] ["", "凯峰", "?", "十二绿洲在那不支持", "购买", "-", "--", "who got"] \N ["one him", "is well with", "过去什么拿不到废话贵姓考前呢里新的"] [] ["w", "h", "q"] ["d", "r", "g", "f", "a", "s", "a", "j", "e", "a"] ["i", "l", "w"] [] [2014-08-12, 2024-01-17, 2025-02-17] [2024-01-19, 2023-12-14, 2023-12-16, 2023-12-10, 2023-12-15, 2023-12-18, 2023-12-17, 2024-01-17, 2023-12-19, 2027-01-16] [2023-12-14, 2024-07-01, 2023-12-19, 2024-01-17, 2027-01-09] [2024-06-30, 2024-01-08, 2023-12-18, 2019-02-16, 2023-01-15, 2023-12-12, 2023-12-17, 2023-12-14] [2025-02-17 00:00:00, 2023-12-14 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02] [] [] [] [2023-12-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-31 00:00:00, 2024-01-31 00:00:00, 2015-04-18 00:00:00] [2026-02-18 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30] [2024-01-17 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00] [2023-12-11 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 2023-12-14 00:00:00, 2025-02-17 00:00:00] [2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00, 2023-12-14 00:00:00, 2025-02-18 00:00:00, 2018-09-05 00:00:00, 2026-01-18 00:00:00] [2023-12-12 00:00:00, 2023-12-19 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00] [9999-12-31 23:59:59, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00] [2024-01-17 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-01-19 00:00:00] +204 1 1 147483648 1451313708 2025-06-18 2023-12-18 2025-02-17 2024-02-18 going at oh that's hey don't like got of 哪一步 p - - 经销总代理拐卖叶子 1 0 65.139100 100.020000 12.132100 -1.200000 2024-01-31 00:00:00 2025-06-18 00:00:00 2024-01-17 00:00:00 2027-01-16 00:00:00 2026-01-18 00:00:00 2025-06-18 00:00:00 2024-07-01 00:00:00 2024-02-18 00:00:00 \N [0, 1, 0, 0, 1] [1, 0, 1, 1, 1, 1, 0, 0, 1, 1] [1, 0, 1, 0, 0, 0, 0, 0, 0, 0] [7, 34, -128, 2, -69, -47, -1, 41, 0, 69] [5, -128, 127] [9, -1, 1, 93, 127] [-128, 127, 30, -69, 127, -1, 1, 74, 127, 46] [19873, 5522, 245] \N [] [] \N [0, 1, -936538102] [2147483647, 697309759, 32679] [-1972317799, 32679, 187904743] [9223372036854775807, -9223372036854775808, -1, 4344658916850647981, 9223372036854775807, 32679, -9223372036854775808, 147483648, -1, -1] [7286303841179209410, -9223372036854775808, -3369886639676660770, 0, -9223372036854775808, 1, -1, -9223372036854775808, 0, 0] [245, 9223372036854775807, 147483648, 147483648, -1862259685, 147483648, -4560899629855702472, -1842364476, 7368178920370379212, -9223372036854775808] [-6428301, 245, -6615830124666046076] [-1, 72554686, -181827568, 1, 0, -2, 1, 0] [-1, 2130000982, 145016378] [1, -1, 8, 0, 951905298, -1, 6, -1] [0, 5, 1912042433, 0, 8, 3, 1510643465, -2, 1322958515, 0] [91.098500000000, 40.123000000000, 40.123000000000, 300.343000000000, 40.123000000000, -1.200000000000, 0.000000000000, 0.000000000000] [81.046400000000, 1.200000000000, 0.000000000000, 94.067300000000, 0.000000000000, 300.343000000000, 58.090000000000, 100.020000000000] [-1.200000000000, 0.000000000000, 300.343000000000, 0.000000000000, 40.123000000000] [1.200000000000, 40.123000000000, 23.195600000000, 50.015700000000, 10.143400000000, 59.135200000000, 0.000000000000, 1.200000000000, 300.343000000000, 40.123000000000] [40.1230, 1.2000, 1.2000, 0.0000, 1.2000, 40.1230, 300.3430, 21.1864, -1.2000, 0.0000] [-1.2000, 7.1674, 0.0000, 17.1837, 16.0746] [1.2000, 97.0896, 40.1230, 1.2000, -1.2000, 48.0367, -1.2000, 300.3430] [40.1230, 100.0200, -1.2000, 89.0230, 76.1338] [47, 300, 1, 300, 82] [] [100, 40, 300, 53, 0] [300, 100, 100, 100, 92, 78, 24, 40, 0, 1] ["--", "果断", "look see then see come my something was time"] ["ok his if with it your one", "have the hey not no were", "?", "as", "他在哪零捌辅佐志彬线程呢个安徽忘不了油漆", "at can't no really hey were from going you", "经过付出过", "ok really all"] [] ["out", "you're at now because right but with because you're", "u"] ["亚科", "in but they", "here as tell no right good from", "so", "my", "d", "弄好战友爸妈", "to on up on good"] ["-", "with look was I'm they as one", "世界拥有售后采购", "?", "are", "号码五万条我不会玩广场提货勇气绿城晚上给把", "say", "l"] [] ["who come then could have you", "-", ""] ["r", "d", "d", "c", "j"] ["y", "i", "v", "y", "z", "h", "l", "w"] ["q", "x", "v", "d", "z"] ["q", "v", "q", "f", "s"] [9999-12-31, 2023-12-12, 9999-12-31, 2001-01-20, 2024-01-09] \N [] [2023-12-15, 2023-12-20, 2023-12-12, 2023-12-11, 2024-01-08] [2024-01-09 00:00:00, 2023-12-13 00:00:00, 2027-01-16 00:00:00] [2023-12-13 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00] [2023-12-12 00:00:00, 2023-12-10 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00] [2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00] \N [2023-12-16 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00] [2025-06-18 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 9999-12-31 00:00:00] [2025-02-18 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2024-01-31 00:00:00, 2024-01-17 00:00:00] [9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00, 2025-02-18 00:00:00] [2024-06-30 12:01:02.123000, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00] [2024-01-09 00:00:00, 2023-12-19 00:00:00, 2018-12-20 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00] [] +205 506399180 1 2135560183 -1488165236 2024-01-17 9999-12-31 2023-12-20 2027-01-09 u 空调 石龙 could of or one want are - 知心打架信息唯一天普 come don't 0 1 0.000000 6.033100 15.165600 100.020000 2026-01-18 00:00:00 2023-12-15 00:00:00 2024-07-01 00:00:00 2023-12-19 00:00:00 2023-12-15 00:00:00 2024-01-19 00:00:00 2023-12-16 00:00:00 2024-08-03 13:08:30 [0, 1, 1, 0, 0, 1, 0, 1, 0, 0] [0, 1, 1, 0, 0] [0, 1, 0, 1, 1] [0, 1, 0, 0, 0, 0, 0, 0] [52, 3, 9, 24, -1] [1, -128, -29, -1, 0, 3, -56, -35, 86, -106] [127, -128, 1, -1, -115] [0, -5, 1, 1, 0, 0, 1, 127, 116, 127] [32767, 4702, 6671, 23662, 245, 25496, 0, 245, 0, -10622] \N [] [0, -12041, -25752] [333053602, 944986853, -2147483648, -1, 1] [] [-1, 2039121822, -586020227, 0, -1857993466, -606213922, 2147483647, 2147483647] [1433720412, 147483648, -1220288450, 2147483647, 147483648, 0, 2110023984, 32679] [3777566199394657565, 1, 254188852, -9223372036854775808, 245, 9223372036854775807, -985259666308586726, 147483648] [-18333529804838854, -1838438757732519208, -783942678, 7738542909239136511, 0] [147483648, 147483648, 9223372036854775807] [32679, 9223372036854775807, 3212342371209844335, -5055253, 0, 9223372036854775807, 251775, -9223372036854775808] [4, -2, 1989037339, 7, -2] [] [1, 1, 0, 0, -1, 0, 0, -1] [-2, 1, -1, 674971137, -1, -2, 0, -1] [57.153900000000, 1.200000000000, 5.195200000000, 29.050800000000, 34.095900000000] [-1.200000000000, 49.178100000000, -1.200000000000, 80.181300000000, 78.119500000000, 11.016900000000, -1.200000000000, 300.343000000000, 13.109900000000, 17.002900000000] [] [-1.200000000000, 40.123000000000, 0.000000000000, 74.011700000000, -1.200000000000, 0.000000000000, 40.123000000000, 29.040300000000, 100.020000000000, 100.020000000000] [-1.2000, 35.1022, 42.1167, 36.0790, 40.1230, 300.3430, 0.0000, 26.1384] [0.0000, -1.2000, 300.3430, 1.2000, 61.0727] [98.1463, -1.2000, -1.2000, 300.3430, 1.2000, 100.0200, 300.3430, 300.3430] [31.1842, -1.2000, 2.1899] [40, 1, 8] [1, -1, 66, 1, 55, 1, -1, 88] [0, 40, 57] [] ["j", "like up hey", "-", "", ""] ["because yeah or no didn't okay some it that there", "her to or they don't", "l", "志光", "-"] ["when right I'll that tell who when see", "配置参数", "", "", "--", "was", "been", "j"] ["for we I'll then i go had hey get yes", "数哟里面拜托一夜这家等级倚天明天找到", "--", "-", "-", "文字", "-", "?"] [] \N ["开车软件园女孩可怕哭泣摄像头就是", "?", "-", "数量", "really"] ["", "I'll one of", "e", "配件贸易通美女鹤壁机器中文李总多少信息", "", "?", "安排一周", "to is they be think ok can were back", "洗手间", "he"] \N ["d", "k", "y", "h", "a"] ["n", "y", "r"] ["k", "g", "v"] [2014-08-12, 2024-06-30, 2024-06-30, 2024-01-31, 2025-02-18, 2023-12-14, 2023-12-10, 2024-02-18] [2023-12-15, 2024-02-18, 2027-01-16, 2023-12-11, 2023-12-18] [2024-01-19, 2024-01-19, 2024-01-08, 2025-02-18, 2024-08-03] [2023-12-09, 2023-12-09, 2025-02-18, 2023-12-12, 2024-01-19, 2024-01-17, 2024-01-09, 2024-01-17, 2023-12-15, 2024-06-30] [] [2024-06-30 12:01:02, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00] [] [2024-01-08 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00] [2024-02-18 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 2023-12-11 00:00:00] [2024-01-17 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2023-12-19 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-01-19 00:00:00, 9999-12-31 23:59:59, 2018-07-05 00:00:00] [2023-12-14 00:00:00, 2016-01-15 00:00:00, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2025-02-18 00:00:00] [] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2023-12-14 00:00:00, 2023-12-20 00:00:00] [2025-06-18 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00, 2023-12-10 00:00:00] [9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00] +206 \N 0 -1812978044 -1422978464 2023-12-17 2026-01-18 2025-06-18 2024-06-30 -- m ? in been you're him did some she her - 六万 as he's as up would here do yeah his 1 0 27.186300 33.098000 1.200000 14.198800 2024-08-03 13:08:30 2024-02-18 00:00:00 2023-01-15 08:32:59.123000 2023-12-13 00:00:00 2025-02-17 00:00:00 2024-02-18 00:00:00 2007-09-23 00:00:00 2023-12-20 00:00:00 \N [0, 0, 1, 1, 1, 0, 0, 0] [] [1, 1, 0, 1, 1] [1, -128, -128, 26, -1, -128, 127, -123] [-41, -73, 1, -85, -1, 127, -128, -1, 30, 127] [118, 1, -1, 127, -128, -1, 1, 104, 0, -128] [127, -51, 127, 5, 29] [-10179, -15080, 22149, 245, 1205] [-19911, -10284, -13558] [-16050, -16108, 7759, 1, 1, -21430, 6300, 17167] [-32269, 29293, 0, 15422, -11096] [] [464970814, -1275870030, 1] [-172630077, -1, -1367350322, -1826961134, -180528692, -1, -2147483648, 1] [] [] [4615208595610381675, 1997632686, 0, 147483648, 245, 245, 1199136229447049892, -6168606101149467443] [-2786261772892404520, 32679, -2166118, -4411517412995706586, 245] [485225036, 0, 147483648, 32679, -812829617, -9223372036854775808, 9223372036854775807, 32679] [] [] [-1168573914, 5, 1958763857, 1, -2, 0, -283187171, 649674960] [1, 0, 1871759078, -2, 1663582458, 0, 476063525, -165635580, 3, 1510971550] [1.200000000000, 100.020000000000, 1.200000000000] [1.200000000000, 40.123000000000, 78.120100000000, 1.200000000000, 300.343000000000] [98.156200000000, 92.183300000000, 0.000000000000] [300.343000000000, 0.000000000000, 26.025000000000, 86.108000000000, 40.123000000000, 300.343000000000, -1.200000000000, 1.002500000000] [] [0.0000, 40.1230, 40.1230, 300.3430, 40.1230] [0.0000, 300.3430, 300.3430] [300.3430, 1.2000, 1.2000, 1.2000, -1.2000, -1.2000, 40.1230, 41.1178] [40, -1, 22, 61, 55, 0, 1, 85] [100, 0, 0, 100, 27, 73, 22, 40] [74, -1, 27, 0, 1, 1, 40, 0] [300, -1, 7, 0, 40, 300, 1, 40] ["c", "原著啊", "do", "士大夫展开大会跟我不下所提供争取一键还原艺术高价", "you"] ["", "why", "背后", "?", "at", "", "昨天", "一年中往年人民红包那种之作东创中层班看他们开发"] [] ["--", "-", "这不是一百万铭岳第三只心连心所有", "方连经过空下热销中顾星人体油画相逢腾讯", "-", "", "?", "b", "才能", "以上返点"] ["had", "at good did your will no i see how we", "to know good mean", "now of hey are is what", "不传被人号码分享平时热情劲不含税", "-", "that's", "?"] ["--", "they", "?", "yeah what", "-"] ["对了", "实话很难说因为先打款珍惜纽曼", "节日月底解决加上", "人家", "参考王晶才会分公司忘不了连成回家", "it", "so", "", "at there him", "--"] ["were", "come one say not we they it's and", "郑旭庆"] ["a", "w", "e", "b", "c", "e", "f", "v"] [] ["x", "n", "f", "x", "s"] ["f", "l", "r", "q", "m", "x", "j", "o"] [2024-01-19, 2023-12-12, 2023-12-19, 2023-12-16, 2024-08-03, 2025-06-18, 2025-02-17, 2024-02-18] \N [2027-01-09, 2019-06-13, 2023-01-15, 2023-12-09, 9999-12-31, 2023-12-13, 2026-02-18, 2027-01-16, 2024-08-03, 2023-12-18] [2024-01-08, 9999-12-31, 2027-01-16, 2023-01-15, 2024-02-18, 2023-12-15, 2023-12-16, 2024-02-18, 2024-01-17, 9999-12-31] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00] \N [2026-02-18 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-12-10 00:00:00] [2023-12-15 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00] [] [2023-12-19 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-16 00:00:00, 2024-01-17 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [2023-12-16 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123000, 2017-11-11 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59.123000, 2027-01-09 00:00:00, 2002-04-24 00:00:00] [2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2012-03-18 00:00:00] [2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-09 00:00:00] [9999-12-31 23:59:59, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [] [2014-08-12 00:00:00, 2023-01-15 08:32:59.123123, 2013-05-21 00:00:00, 2025-02-18 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00] +207 -2147483648 0 91624372 785788426 2024-01-19 9999-12-31 2024-02-18 2024-01-17 so the the can be okay this yeah want 北京朝阳区今天为了最后好友方面动物大概科汇优先 请你腾龙天浩平均高压回龙 是哦 m your -- - 0 0 -1.200000 100.020000 300.343000 77.028300 2025-06-18 00:00:00 2024-02-18 00:00:00 2023-12-09 00:00:00 2023-12-16 00:00:00 2026-01-18 00:00:00 2023-12-18 00:00:00 2027-01-09 00:00:00 9999-12-31 00:00:00 [] [1, 1, 0, 1, 0, 1, 1, 1, 1, 0] [1, 0, 1, 1, 1] [1, 1, 1, 1, 0] [-124, 1, 0, -128, 1] [-1, 62, -128, 17, 0, 9, 9, -1] [1, 1, -128, 76, 1, 57, -128, -128] [] [0, 32096, -27728, 0, -3517, 31756, 1, 1, -2802, 19394] [] [-32768, 2208, 13329, 2610, 17897] [-12990, 10045, -32768, -1, 0, 0, -32768, -32768] [2147483647, 650764281, -1, 971886704, -2126236321, -1108333312, -2091536548, 1, 0, 0] [498869509, 2147483647, -1782132957, 2147483647, -984509658, 1, 1183013744, -1199945414] [] [147483648, 1, 2147483647, 0, -455885265] [32679, -9223372036854775808, -9223372036854775808, 245, 9223372036854775807, 1840536961, 245, -1] [-9223372036854775808, 5430002266463450877, 32679, -1882550876564627201, 1, 245, 245, -834685220348324631] [34041, 1, 5797705528167560793, -5900085, 866804763856138882, 147483648, -3609925862298697298, 32679, -2698051217104708491, -6250498774556462569] [1, -4651414, -122853499, 2079205508526186762, 0] [0, -2, -2, 0, -2, -1985414272, -2, 0] [-1373909019, -1, 0, 202170894, -995103686] [] [-684593594, -2, 1] [] [300.343000000000, -1.200000000000, 1.200000000000, 0.000000000000, 40.123000000000, 94.179400000000, 100.020000000000, 29.197800000000] [] [300.343000000000, 70.107700000000, 39.052100000000, -1.200000000000, 300.343000000000, 35.000200000000, 43.081500000000, 100.006300000000] [26.0809, 0.0000, 1.2000] [] [99.1767, 0.0000, 0.0000, 40.1230, 48.1708, 0.0000, -1.2000, 40.1230] [] [7, 0, 1, 90, 59] [0, 300, 1] [300, 44, 1, 0, -1, 100, 300, 1, 1, 0] [40, 40, 0, 40, 8, 33, 300, 100] ["呵呵呵房子导演对吧谁在", "can your as out ok like about as didn't", "I'll in tell to as or but some out when"] [] ["我在郑州现在", "his really yeah go if the you", "back something all", "g", "n", "小键盘智博汇集见你饲料王峰副总经营不含税签名", "河南总经销没有到货据对你杂报价格能早点零售黄经理小曹", "机会做梦商家", "m", "--"] [] ["p", "--", "g"] ["now", "r", "i"] ["--", "good as it's", "?"] ["out", "a", ""] ["b", "p", "w", "i", "s", "h", "u", "f"] ["p", "l", "a", "d", "l", "h", "y", "d"] ["z", "h", "d", "r", "i", "n", "v", "y", "w", "y"] [] [2024-02-18, 2026-01-18, 2023-12-17, 2024-07-01, 2023-12-10, 9999-12-31, 2023-12-11, 2026-01-18, 2024-01-08, 2023-12-11] [2025-06-18, 2023-12-09, 2023-12-17, 2027-01-09, 2027-01-09] [2011-02-23, 2023-12-18, 2014-08-12] [] [] [2027-01-16 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00] [2025-02-17 00:00:00, 2026-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-13 00:00:00, 2024-01-08 00:00:00, 2024-08-03 13:08:30, 2023-01-15 08:32:59, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00] [2024-01-17 00:00:00, 2023-01-15 08:32:59, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-09 00:00:00, 2024-08-03 13:08:30, 2024-01-09 00:00:00, 2023-12-13 00:00:00] [2023-12-16 00:00:00, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00, 2024-08-03 13:08:30] \N [2014-08-12 00:00:00, 2023-12-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2026-01-18 00:00:00] \N [9999-12-31 00:00:00, 2017-05-03 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-01-19 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2023-12-11 00:00:00] +208 147483648 -2147483648 -1079149238 997800124 2024-02-18 2027-01-16 2023-12-13 2024-01-09 f \N here up come okay well are here that's at yeah i 小苏张先生福利多少见过 ? 意义词库上次键盘不忙没时间况且四川多家你有智慧打电话 关闭你吃法方法 1 1 300.343000 -1.200000 1.200000 0.000000 2023-12-13 00:00:00 2024-02-18 00:00:00 9999-12-31 23:59:59 2023-12-16 00:00:00 2026-02-18 00:00:00 2024-01-31 00:00:00 2023-01-15 08:32:59.123123 2024-02-18 00:00:00 [] \N [] [1, 0, 0, 1, 0, 1, 0, 0, 0, 1] [127, -45, 1, 1, 0, 9, -1, -43, -95, 127] [2, -1, 127, -128, -47, 0, -17, 0, -121, 3] [-69, 127, -128, -128, -128, 1, 127, 1] [127, 1, -124, -42, -1, 0, 127, 127] [4584, -1, -8381, 1, 19011, 13129, 0, 28951, 1, -32768] [-32768, 0, 1, 27325, -7235] [245, -32768, 10398, 0, -1] [] [359076183, 147483648, 1, 32679, 32679, 0, 1910253631, -2147483648, 89679507, 2147483647] [32679, -1607550237, -1315104280, -1376702402, -1626259995] [0, -2147483648, -1, -1, 201296134, -1, 1100676565, -1546118320] [32679, -2147483648, 32679, 2147483647, -695448962] \N [5554023344304130097, -5070353081554061730, -2079055197, 0, 1] [928275, 1, 1, 32679, 1136786350] [-4986865, -1, 245] [2, 2013889272, 4, -2, 671495824] [0, 5, 232904621, -2, 4] [0, -1, 0] [1, 1, 1] [0.000000000000, 40.173000000000, 40.123000000000, 40.123000000000, 39.041600000000] [89.176500000000, 84.038800000000, 300.343000000000] [54.124400000000, 40.123000000000, 78.072600000000, 300.343000000000, 8.019700000000] [-1.200000000000, 100.020000000000, 20.068300000000, 300.343000000000, 78.154500000000] [-1.2000, 0.0000, 1.2000, 0.0000, 100.0200, 78.1175, 34.0017, 40.1230] [300.3430, 48.0723, -1.2000, 0.0000, 62.0229, 100.0200, 1.2000, 6.1202] [91.1425, -1.2000, 68.0592, -1.2000, 29.0822] [] [0, 40, -1, 38, 300, 100, 71, 300, 40, 100] [] [21, -1, 100, 300, 1, 59, 0, 40] [1, 100, 40] ["联硕不做真正以以上的", "图站你说杀了影子是的南风走在还好吧签名", "didn't here", "i", "?", "want", "手机", "人工湖"] ["-", "?", ""] [] ["q", "事务所加上都有不让导演伴奏许愿树", "you", "", "-"] ["-", "?", "out and", "", "?", "j", "because", "反应"] ["", "with ok get go want was get", "?", "?", "--", "about mean here something your", "from", "这三大"] ["--", "去过", "?", "-", "--", "努力列车表光纤平均天数感受充满", "s", "know", "on be she had like not something well why", "安信海天诺顿气人而是好了粘贴"] [] ["m", "e", "r"] ["o", "s", "o"] [] [] [2023-12-12, 2024-01-31, 2024-01-17] [2023-12-18, 2024-02-18, 2027-01-16, 2023-12-16, 2023-12-17] [2023-12-16, 2023-12-13, 2026-01-18, 2023-12-12, 2025-06-18, 2024-01-17, 2025-06-18, 2024-06-30] [2025-02-18, 2027-01-16, 2014-08-12, 2023-12-12, 2023-12-10] [2025-06-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00] [] [] [2023-12-14 00:00:00, 2026-01-18 00:00:00, 2023-12-11 00:00:00] [2024-01-08 00:00:00, 2024-01-08 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00] [2014-08-12 00:00:00, 2026-02-18 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00] [2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-18 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00, 2024-01-08 00:00:00] [2025-06-18 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00, 2025-02-18 00:00:00] [2024-07-01 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2018-02-05 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00, 2027-01-16 00:00:00] [2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00, 2023-12-13 00:00:00] [2024-01-08 00:00:00, 2026-01-18 00:00:00, 2024-07-01 00:00:00, 2011-12-15 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123123] +209 0 0 1332557322 1630863954 2023-12-14 2025-06-18 2024-01-31 2025-02-18 不可能期盼夜市我能杀毒 也冲击 \N ? 谢谢翔龙稍微今晚什么的人生准确有点忙总数 how ? 0 1 40.123000 \N 87.135600 300.343000 2023-12-14 00:00:00 2024-02-18 00:00:00 2023-01-15 08:32:59.123000 2024-01-09 00:00:00 2023-12-09 00:00:00 2024-02-18 00:00:00 2027-01-09 00:00:00 2025-06-18 00:00:00 \N [0, 1, 0, 1, 0, 0, 0, 0] [] [1, 1, 0, 1, 1, 1, 0, 1, 1, 1] [0, 0, 0, -128, 2] [10, 1, 4, 23, 127, 12, -1, 48, 0, 127] [127, -128, -104, 1, 0, 94, 63, -1] [-1, 2, 0, 1, -1] [-1, -2058, 32767] [29790, 25835, -1, 31436, 0] [3374, -21964, 23682, 0, 245, 20647, -1, -1, 7095, -1938] [22455, -20776, -32768, -1, 20266, 32767, 7742, 32767, -24082, -14001] [-1274624786, 32679, -2147483648] [147483648, -2147483648, -195165989, -953532853, 1982745190, -1, 1387801749, 2147483647, 1331458551, 0] [147483648, 147483648, 0, -2004555330, 32679, 0, 32679, -1] [32679, 980788168, -824727926] [6817069615849418439, 9223372036854775807, 147483648, -686955147, 32679, 1, 1, 0, -9223372036854775808, 6479636724654817156] [245, 9223372036854775807, 1, 5414290397036897287, 5719376242291452432] [-3030167, -742764326, -9203078808845140004, 245, 245] [0, 1, 5425123] [1, 1951595723, 0, 5, -2084137657, -1, 0, 0] [0, 0, -1, 0, -1727696415] [1, -1, -2, 1, -1, 6, 1, 0, -1, 1] [] [100.020000000000, 100.020000000000, 0.000000000000] \N [40.123000000000, 1.200000000000, 300.343000000000] [-1.200000000000, 40.123000000000, 7.040000000000, 40.123000000000, 40.123000000000, 300.343000000000, -1.200000000000, 56.161700000000] [-1.2000, -1.2000, 1.2000, 40.1230, 1.2000, -1.2000, 100.0200, 100.0200] [4.1310, 1.2000, 1.2000, 73.0038, 0.0000, 0.0000, 69.0057, -1.2000, 21.0068, 9.1452] [-1.2000, 0.0000, 40.1230] [-1.2000, 40.1230, 48.0895, 40.1230, 300.3430] [2, 40, 89, 0, 0] [300, 45, 0, 0, 300, -1, -1, 96] [100, 89, 300, 1, 81, 40, 42, 20, 1, 37] [0, 23, 26, 11, 300, 20, -1, 34] ["he if when can", "what I'm i at with at my so can't", "?", "it's", "?", "--", "?", "管理"] ["?", "p", "the i they I'll when no this just", "?", "中小型三级看待新一代本科"] ["all", "and going going yes they can on no know up", "r"] ["数码", "--", "be"] ["--", "one", "her me that's as hey time about him", "谢晓娟侯玉见过面改动", "x"] ["--", "--", "--", "集群多少钱", "?", "分别核算首创人民路可笑切换", "这一说", "not who mean up now you was"] ["his got have been well been go some", "稳定性自己的九九长光海涛", "as didn't who your"] [] ["k", "e", "s", "b", "m"] [] ["d", "p", "i"] ["v", "s", "i", "e", "r", "i", "w", "u"] [2024-01-08, 2027-01-16, 2025-06-18, 2023-12-10, 2023-12-18] [2024-02-18, 2024-02-18, 2024-07-01] [2025-02-17, 2023-12-13, 2024-01-09, 2024-08-03, 2023-01-15, 2024-02-18, 2023-12-15, 2027-01-16] [2024-02-18, 2024-02-18, 2023-01-15, 2023-12-11, 2026-02-18] [2025-02-17 00:00:00, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00] [2026-01-18 00:00:00, 2023-12-11 00:00:00, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 2023-12-15 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00] [2027-01-09 00:00:00, 2024-06-30 12:01:02, 2023-01-15 08:32:59, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00] [] [2023-12-20 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2027-01-16 00:00:00] [2023-12-14 00:00:00, 2019-10-12 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [] [2023-12-13 00:00:00, 2023-12-14 00:00:00, 2023-12-18 00:00:00] [2023-12-19 00:00:00, 2024-08-03 13:08:30, 2024-01-09 00:00:00] [2024-01-31 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00] [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2015-05-08 00:00:00, 2009-11-22 00:00:00] +212 -1 809104002 32679 -1811461282 2024-01-31 2027-01-09 2027-01-16 2024-02-18 心理学 ? - up are her of 腾龙 you're 0 0 1.200000 \N 40.123000 -1.200000 2008-04-24 00:00:00 9999-12-31 00:00:00 2023-01-15 08:32:59.123000 2024-07-01 00:00:00 2027-01-16 00:00:00 \N 2026-01-18 00:00:00 2023-12-20 00:00:00 [1, 0, 1, 1, 1, 1, 1, 0] [0, 1, 0, 1, 0] [0, 1, 1, 0, 1] [0, 1, 0, 0, 1] [] [1, -128, 0, 1, 9, 7, 0, -128, 4, -1] [] [] [-16085, 0, -32768, -32768, -1470, 1, -20416, 245, 31823, 17707] [-32768, -1, 23932, -17484, -32768, 2350, 24000, 32767, 7914, 10155] [1, 245, 14702, 5140, -10741, 29204, 25532, 11445] [] [1, -1, -2147483648, -154052715, -1434783947] [147483648, 0, -1] [2147483647, 1, -142386210, 1061492030, -101689933] [-1200852028, -775860539, 0, -1, -915734344] [0, -2557686, 9223372036854775807, -1792656298144538330, 1] [2949103884726325197, -1062944591009464591, 32679, 0, 4052672372382474113, -6679849581953205783, -1294897938, 9223372036854775807, 0, -9223372036854775808] [-1, -7927082304553415186, -1, 4544486788403353940, 32679, 147483648, 245, 1, 147483648, 147483648] [32679, 3921767237304639852, 147483648, -889974006, -1, 245, 32679, 32679] [950869136, -2, 1281831668, 0, -1401517204] [1, -1, 0, 1, 1] [-1, 0, 1, 4, -1] [0, 1, 6, 4, -2, -2, -2, -1] [75.047000000000, 100.020000000000, 1.200000000000, 100.020000000000, 100.020000000000, 0.000000000000, 0.000000000000, 67.111700000000] \N [46.110700000000, 56.091500000000, 40.123000000000, 87.092600000000, 1.200000000000, -1.200000000000, 300.343000000000, 60.165100000000] [41.128800000000, 40.123000000000, 100.020000000000, 100.020000000000, -1.200000000000, 40.123000000000, 17.174100000000, 27.002600000000, 0.172400000000, 98.199600000000] [300.3430, 0.0000, 1.2000] [] [0.0000, 43.0322, 0.0000] [68.0703, 0.0000, 1.2000, 300.3430, 40.1230, 40.1230, 14.1608, 60.0300, 22.0919, 26.0487] [1, 0, 37] [1, 1, 55, 300, -1, 20, 19, -1, 40, 1] [-1, 1, 100, 41, 0, 0, 67, 30] [68, 100, 3, 40, 40, 0, 40, 300] ["did about go didn't how back who but to got", "got", "", "-", "he's", "大公司短语哪呢个平志伟哥们", "--", "not"] ["he", "", "不对战友"] ["up you this", "-", "--"] ["?", "们认识培新新国发票牡丹公司柯艳琦元月", "back", "新鑫", "零捌做不到两者考试", "得劲", "go all then", "-"] ["看是回首不便", "-", "充满"] ["z", "信号昨天占到清理被授予目前这种大领导", "i how the with a", "肆仟", "the how", "some back oh at now he's why was how go", "?", "--"] ["?", "-", "神仙"] [] ["h", "v", "z"] ["g", "d", "z", "g", "p", "j", "z", "j", "s", "y"] ["v", "j", "f", "d", "j"] [] [2026-02-18, 2025-02-18, 2026-01-18] [2025-02-17, 9999-12-31, 2027-01-16, 2023-12-14, 2023-12-13] [] [2026-02-18, 9999-12-31, 2024-01-19, 2027-01-16, 2024-02-18, 2023-12-12, 2024-08-03, 2024-01-17, 2024-07-01, 2023-12-17] \N [] [2023-12-16 00:00:00, 2014-08-12 00:00:00, 2025-02-18 00:00:00] [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59, 2007-09-25 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00] [2024-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2016-03-17 00:00:00, 2023-12-12 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00] [2024-01-19 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00] [2023-12-19 00:00:00, 2023-12-11 00:00:00, 2015-05-03 00:00:00] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-16 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2024-01-09 00:00:00] [2023-12-15 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00] +213 -1 -51645253 147483648 -2147483648 2023-01-15 2024-01-09 2023-12-20 2023-12-17 组装总代开会有一天不难明白合作 朱晓明 陈文杰 看不到 - him like so right got would want this ? 兆文 1 0 55.095600 8.146300 1.200000 8.177100 2025-02-17 00:00:00 2023-12-11 00:00:00 2023-12-13 00:00:00 2008-10-07 00:00:00 2023-12-18 00:00:00 2023-12-20 00:00:00 2025-06-18 00:00:00 2024-02-18 00:00:00 [0, 1, 0, 1, 0] [0, 0, 0] [0, 1, 1, 0, 1] [1, 0, 1, 1, 0, 1, 0, 1] [-88, -1, -128] [0, -1, 0, -1, -128, 49, 3, -1] [] [0, 44, 11, 48, 1, 4, 99, 0, -33, 101] [32767, -1, -16107, 22744, -19760, 31431, 245, -12250] [] [-30096, 8512, 21168, -22867, 245, 10834, -28523, -29796] [1, -12229, 3928, 0, 32767, -26215, -32768, 245, 245, 0] [-2147483648, 571513256, 147483648] [-428282037, -1, -1, 0, 147483648, -1, 147483648, 2021320213] [296406138, 32679, 32679] [1, 388247732, 31518119, 731858159, 32679] [3601803864336312705, -9223372036854775808, -2687269, -3715109515585471679, 1111690837914993278] [2125397, 2336360138670098706, -1] [] [-2888108947684439743, 2522107958340050465, 0, 1677018244, 9223372036854775807, 1, 8373950866633327622, 245, 245, 9223372036854775807] [] [0, -2, -2] [] [115923176, -1, 8, -7563852, 1] [-1.200000000000, 100.020000000000, -1.200000000000, -1.200000000000, 42.063100000000] [40.123000000000, 70.067600000000, 0.000000000000, 300.343000000000, 40.123000000000] [] [40.123000000000, 0.000000000000, 89.139400000000, -1.200000000000, 0.000000000000] [40.1230, 300.3430, 1.2000, 40.1230, -1.2000, 38.0767, 82.0813, 80.1057] [1.2000, 100.0200, -1.2000] [50.0547, 100.0200, -1.2000, 1.2000, 1.2000] [300.3430, 85.1997, 300.3430, 40.1230, 100.0200, 300.3430, 40.1230, 0.0000, -1.2000, 100.0200] [100, 40, -1, -1, -1] [1, 300, 13, -1, 300] [0, 34, 43] [62, 40, 58] [] \N ["-", "改掉五号邮箱", "say", "--", "?", "明华", "真伪好友", "at"] ["there", "?", "三业也冲击增值", "with this out on had about a", "?", "go", "think I'll back i it on don't how tell", "后面"] ["--", "群英", "长信图腾参数手册"] ["why then some", "?", "早点", "", "would for say come some he's I'll me some"] ["--", "李总", "v", "综合", "--", "高一些", "out", "", "稳定瑞昌怎么顺风下面邮箱大量现货你杀时间高价", "s"] ["排列", "t", "最新第二最大湖北网易"] ["t", "f", "i", "s", "x", "a", "p", "q"] ["j", "t", "b", "y", "z", "y", "o", "z"] ["w", "j", "x", "e", "o", "c", "e", "i", "j", "m"] ["c", "c", "y", "w", "j", "p", "v", "f"] [2024-01-31, 2024-02-18, 2011-07-16, 2024-08-03, 2025-02-18, 2023-01-15, 2026-02-18, 2026-01-18, 2025-06-18, 2025-02-18] [2023-12-13, 2026-01-18, 2014-08-12, 2023-12-17, 9999-12-31, 2024-01-09, 2024-02-18, 2024-02-18, 2024-06-30, 2026-01-18] [9999-12-31, 2024-01-31, 2023-12-19, 2008-12-17, 2025-02-18] [2023-12-16, 2023-12-17, 2023-12-17] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2007-02-10 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-09 00:00:00] [2024-07-01 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00] [2015-11-16 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00] [2014-08-12 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-14 00:00:00, 2023-01-15 08:32:59.123000, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00] [2007-01-20 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2011-06-12 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00, 2023-12-13 00:00:00, 9999-12-31 23:59:59] [2023-12-19 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2012-08-14 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2027-01-16 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00] +214 -1 1960092740 0 1 2024-07-01 2014-08-12 2023-12-12 2025-06-18 计算赞美诗供货商办事处 开发做出也在不能汇集三星下了 呼喊跟我方向备注王枫路上涨价 \N 喜剧 - ? 潇湘发布景泰周星驰不会副本备份文件自学考试能早点丁老师 0 1 1.200000 1.200000 300.343000 40.123000 2023-12-12 00:00:00 2023-12-18 00:00:00 2025-06-18 00:00:00 2024-01-19 00:00:00 2026-01-18 00:00:00 2024-02-18 00:00:00 2023-12-13 00:00:00 2024-01-19 00:00:00 \N [] [0, 0, 0, 0, 0, 0, 0, 1, 0, 0] [0, 1, 0] [1, 0, -1] [5, 1, -87] [-119, -112, 3] [-117, 2, -28, 8, 127, -128, -1, 0, -128, 7] [-23337, 10108, 23326, 245, -6334, 1, -987, 1, -13050, -32768] [-19400, -4172, 32767] [-12506, -1, -32768, 0, 32767, -32768, -32768, -10313, 0, 32767] [3750, -31631, 25208, 29936, 504, -2930, 8459, 14292] [0, 32679, 86256343, 1980120954, 2147483647, 32679, -1580777826, -665886353] [1, 2147483647, 2147483647, -2147483648, -2147483648, 32679, -2147483648, 2007689711] [] [147483648, -333685487, -2147483648, -1, 32679, 32679, 147483648, -690640920] [-9223372036854775808, 0, -1] [0, -2530528132136074110, -1, 147483648, 821865279640017994, 0, 1153946, -4936926085533697920, 245, -6863738664623121932] [32679, 7495651056743177995, 147483648, -674902272, 663549961, -1505906, 147483648, -3374815166305384747, 258815227929855151, -8901397586982541867] [-1, -9223372036854775808, -8309031016747589503, 705606093, -1, 1, 245, 1, 1, 32679] \N [1, -1, -2, -2, 1703374820, 8, 1, -1, 1886895470, 0] [1, -1, -2, 0, 0, 7, 1, -1, 0, -2] [-2, 1, 0, -1, 0, 0, -2, -1388083499] [93.198200000000, 100.020000000000, 0.000000000000, 52.085200000000, 40.123000000000, 40.123000000000, 57.055100000000, 40.123000000000, 100.020000000000, 40.123000000000] [40.123000000000, 300.343000000000, -1.200000000000] [1.200000000000, 84.180500000000, 1.200000000000] [42.164700000000, 100.020000000000, 89.163900000000, 100.020000000000, 8.191800000000, 64.186900000000, 300.343000000000, 70.015000000000, 92.062100000000, 40.123000000000] [7.1782, 21.1843, 1.2000, 40.1230, -1.2000] [] [1.2000, 50.1978, 0.0000] [0.0000, 0.0000, 0.0000, -1.2000, 43.0654, 8.0868, 1.2000, 100.0200] [300, 91, 95, 100, 40] [0, 1, 1, 25, -1] [57, 300, -1, 0, 40, 1, 40, 52] [] [] ["get", "can't", "--", "had", "", "b", "could", "办完", "can", "if because come on could can who"] ["国际企业见过面长信代理价格", "光纤客户七匹狼这上面", "o", "?", "would", "y", "did know", "市场淀粉"] ["同学一键还原王枫咩女王青", "--", ""] ["-", "", ""] ["", "景峰", "-", "教练哈哈好卖快递原装张先波写上限价", "-", "from", "did i on were", "it's"] ["she", "", "did", "was up hey", "would my yeah why yeah me the that"] ["had", "多星资料", "过来", "don't", "信息化"] [] ["t", "p", "t"] [] ["v", "m", "u", "i", "g", "h", "b", "w", "b", "s"] [2024-02-18, 2014-08-12, 2024-01-17] [2024-07-01, 2025-06-18, 2023-12-09, 2023-12-20, 9999-12-31] [2024-07-01, 2023-12-11, 2023-12-09] [2024-01-17, 2023-01-15, 2023-12-20] [2023-12-16 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [] [2023-01-15 08:32:59, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00] [2023-12-19 00:00:00, 2024-07-01 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00] [2023-12-15 00:00:00, 2023-12-17 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00] [2024-01-09 00:00:00, 2023-12-15 00:00:00, 2027-01-09 00:00:00, 2023-12-20 00:00:00, 2024-01-08 00:00:00] [2027-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2005-07-08 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 2003-10-04 00:00:00, 2023-12-13 00:00:00, 2023-12-14 00:00:00, 2023-12-12 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2024-01-08 00:00:00] [2025-02-17 00:00:00, 2023-12-12 00:00:00, 2014-08-12 00:00:00, 2024-01-08 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-17 00:00:00] [2023-12-18 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00] +215 -2147483648 147483648 924618412 -2147483648 2024-02-18 2023-12-19 2024-01-09 2023-12-12 想象 双飞燕 no not just all back ? 睡觉马上两者明辉很难说都发生叁万 about they her \N 1 300.343000 -1.200000 44.158200 28.135500 2026-01-18 00:00:00 2025-02-18 00:00:00 2024-08-03 13:08:30 2023-01-15 08:32:59.123000 9999-12-31 00:00:00 2023-12-09 00:00:00 2023-12-12 00:00:00 2024-01-17 00:00:00 [1, 0, 1, 0, 0, 1, 0, 0] [0, 0, 1, 1, 1, 0, 1, 1, 0, 0] [1, 0, 1] [0, 1, 0, 1, 0, 1, 0, 0] \N [127, -58, 124, 65, -128, 127, 1, 0, -18, 1] [-128, -1, -1, 127, -1, 19, -1, -1] [1, 127, -98, 127, 0] [-30795, -26622, 1, 84, 26787, 30681, -2679, 24425, -17669, 1] \N [-10050, -1, 12024] [-32768, -1, -8029] \N [2147483647, 0, 1, -1474786951, -1495851146, 147483648, -1, -700181428, -1, -502504558] [2147483647, 846764533, -1, -2147483648, 147483648, 117907561, -1225618994, -2147483648, 782922162, 1] [-2055276214, 32679, 0, 541844929, 1, 147483648, 1, 0] [6040393, 245, 0, -2698100, 245, 6848689, -9223372036854775808, -1710479, 8703697944843603526, 0] [1381429079763293901, -8150700633666696545, 613555868144968085, 32679, 147483648] [9223372036854775807, 245, -9223372036854775808, -1, 2506159269920230191] [90712849313334451, -9223372036854775808, -1] [2, -2, 1] [-2, -2, 309315432, 1, 0, 1, -1, -2] [0, -2, 0] [-2, 3, -1] [23.016600000000, 1.200000000000, 1.200000000000] [40.123000000000, -1.200000000000, 34.171300000000, 0.000000000000, 40.123000000000, 40.123000000000, 0.082400000000, -1.200000000000] [300.343000000000, 48.037500000000, 0.000000000000] [] [97.0013, 300.3430, -1.2000] [53.0178, 1.2000, 1.2000, 1.2000, 55.1174, 36.1897, 8.1318, 1.2000] [40.1230, 100.0200, 2.0298, -1.2000, 96.0614, 300.3430, 22.1805, 100.0200] [0.0000, 40.1230, -1.2000, 100.0200, 1.2000, 0.0000, 0.0000, 40.1230, 90.0057, -1.2000] [] [100, 1, 89, 40, 100, 0, 40, 40] [1, -1, -1, 45, 300, 1, 9, 3] [300, 1, 1] [] ["ok", "--", "something"] ["--", "", "-"] [] ["", "写在", "--", "one", "好久", "-", "know", "套件一般", "许文", "山下一千多开拓宋玉慧不多制造跃民鸿锐外地"] [] ["", "t", "--", "-", "-", "w", "", ""] ["a was come her he's", "", "time and some you", "", "-", "with mean oh didn't she or we okay would as", "--", "害怕格瑞真伪在不在目前不方便查到写上王晶勾丽娜"] \N ["s", "g", "d", "u", "u"] ["u", "z", "z", "b", "w", "e", "l", "y", "u", "d"] ["p", "f", "c", "y", "f"] [2024-01-19, 2027-01-16, 2023-01-15, 2024-02-18, 2026-02-18, 2024-01-17, 2004-05-28, 2024-01-19] [2027-01-16, 2024-06-30, 2024-07-01, 2023-01-15, 2023-12-19] [2024-08-03, 2023-12-16, 2023-12-20] [2024-06-30, 2023-12-16, 2023-12-14, 2023-12-15, 2024-01-09, 2024-01-08, 2024-08-03, 2023-01-15, 2026-01-18, 2024-02-18] [2023-01-15 08:32:59, 2023-01-15 08:32:59, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] [2023-01-15 08:32:59, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00] [2026-01-18 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00] [2024-07-01 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2017-06-28 00:00:00, 2003-04-23 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2025-02-17 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2024-01-19 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00] [] [] [2023-01-15 08:32:59.123123, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 2023-12-11 00:00:00] [2023-12-09 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] [2024-01-09 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] +218 2147483647 1 -1941517862 0 9999-12-31 2023-12-13 2025-06-18 9999-12-31 think can't here could can't because can't she why she come if mean s f would -- 1 0 1.013000 54.120100 300.343000 7.160400 2023-12-20 00:00:00 2024-08-03 13:08:30 2024-01-17 00:00:00 2023-12-14 00:00:00 2023-12-13 00:00:00 9999-12-31 23:59:59 2023-12-13 00:00:00 2023-12-10 00:00:00 [0, 0, 1, 0, 1] [1, 0, 0, 0, 1, 0, 1, 0] [1, 1, 1] [1, 0, 0, 0, 1, 1, 1, 1] \N [0, -8, 127, 127, -128, 124, 127, 90] [1, -128, 0, 76, -1, -1, 0, -128] [-67, 0, 38, -49, -1, 127, -1, -1, 6, 1] [-4596, 0, 10448, 13830, 0] [-1973, -1, -8546, 32767, -1, 0, 25820, -13530] [32767, 245, 24039, -1, 245, 245, -14263, -31738] [0, -1794, 245, -24652, 32767] [-1303541585, 147483648, 147483648, 32679, 1737205159, -1, -1107446198, 1] [2024156514, -1, 0, -723999912, 68991303, -598854984, -1172135523, 0] [32679, 0, 1, 1, 1, -841524483, -737543268, 147483648] [] [] [-2079547, 9223372036854775807, -1, 9044701561220697308, -896070818] [-2846919138930634946, 0, 147483648, 8795097708137512783, 4631318551674258645] [1, 0, 5496450, 4333098286928877234, -9223372036854775808, 2895244904105833363, 1, 245, 147483648, -3562223] [0, 0, 4, 1, 788767671, 1, 1, 1, 1818047013, 0] [] [2, -1, 0, -1, 6, -1, 9, -1] [1, -1573352962, -2, 1, -2, 4, -2, -1] [59.047100000000, 10.188100000000, 72.171100000000] [-1.200000000000, 34.092100000000, 100.020000000000, 40.123000000000, 300.343000000000] [41.148500000000, 100.020000000000, 40.123000000000] [79.022300000000, 63.040400000000, 0.000000000000, 300.343000000000, 300.343000000000, 300.343000000000, 40.123000000000, 1.200000000000, 64.123400000000, 1.200000000000] [100.0806, 300.3430, 300.3430, 74.0705, 32.1260, 40.1230, 300.3430, 100.0200] [40.1230, 29.1661, 1.2000, -1.2000, 0.0000] [86.0504, 40.1230, 0.0000, 300.3430, 3.0858, 300.3430, 91.0268, 1.2000, 1.2000, 42.0743] [34.1381, 100.0200, 29.0154, 95.0864, 67.1351] [20, 95, 0, 0, 300, -1, 0, 0, 81, -1] \N [300, 0, 40, 300, 51, 26, 5, 50] [-1, 40, -1] ["给你", "s", "宏运", "网页简介考核你早几天回来吧瑞贺着急福利免费", "--"] ["", "交货不会不问神仙", "-", "v", "准备消失艾蕊"] ["视讯", "out", "", "-", "见过面", "back come do it's all", "", "?"] ["", "?", "--", "", "mean just"] ["me", "--", "小孩", "me", "have are now when will if I'll can't going", "稳定性代理商号发给河南分区电池验货单不像没有办法山特", "?", "不开改改中成瑞贺老人家若虚维泽"] ["-", "", "一起"] ["", "all really then come see", "can"] [] [] ["o", "u", "b", "r", "e"] [] [] [2023-12-20, 2026-01-18, 2003-07-12, 2023-12-13, 2025-02-18, 2025-02-18, 2024-01-31, 2024-02-18] [] [2023-12-13, 2024-01-09, 2024-06-30] [2025-06-18, 2024-08-03, 2023-12-10, 2023-12-14, 2023-12-15] [2027-01-16 00:00:00, 2023-12-12 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2026-01-18 00:00:00] [2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [9999-12-31 23:59:59, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00] [2023-12-14 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00] [2023-12-10 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00] [2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] [] [2023-12-10 00:00:00, 2023-12-16 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00, 2023-12-15 00:00:00] [2024-07-01 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-31 00:00:00, 2008-12-22 00:00:00, 2023-12-18 00:00:00] [2024-01-17 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2024-01-08 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 2025-02-18 00:00:00] [2023-12-11 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00] [2027-01-09 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00] +219 -1421211883 1 32679 694965967 2024-01-31 2024-02-18 2023-12-18 2024-01-08 - x ? had - in want how -- 亲戚品放标底 \N 0 300.343000 100.020000 40.123000 1.200000 2023-12-09 00:00:00 2024-07-01 00:00:00 2024-07-01 00:00:00 2023-12-20 00:00:00 2025-02-18 00:00:00 9999-12-31 00:00:00 2023-12-16 00:00:00 2025-06-18 00:00:00 [0, 1, 0] \N [] [1, 1, 1, 0, 1, 0, 0, 0, 0, 1] [] [0, -1, -128, 0, -111] [-40, 24, -128, 94, 2, 38, -47, -1] [1, 0, 0] \N [13266, -12844, 31505, -19300, 0, 0, 245, 14753] [-1, -1, -32768, 9944, -18643, 263, -1, -27670, 0, 32767] [] [1713379429, -2147483648, 1, -1, -1, 1145748792, 0, -2147483648, 433657698, -1490964520] [2147483647, 0, 2147483647, 2147483647, 1, 2147483647, 243522967, 1050024295, -2079891327, 1619598946] [654237149, 2147483647, -211876542, -1861050325, 2147483647] [1795486611, 2147483647, -705712281, 1125174855, 147483648, 1218844583, 907809809, -1932048686, 902917490, -1779022368] [9223372036854775807, 147483648, -9223372036854775808, -5047988509999574379, 9223372036854775807, -1, 4010360, 147483648] [32679, 974982848529663849, 1, 245, -1] [9223372036854775807, 245, 245, 6116756779440291998, -1, 0, 2203784538362283027, 245, 9223372036854775807, 1] [4770114, 1, -9223372036854775808] [] [-2, -580815659, 1, -729541482, 1, 0, -2, -1] [-1, 1, 1, -1, 1] [5, 1, -1, -1, -930717796, 1, 0, 2] [300.343000000000, 40.123000000000, 0.000000000000, 33.197700000000, -1.200000000000] [-1.200000000000, 40.123000000000, -1.200000000000] [100.020000000000, 66.064000000000, 64.145600000000, 0.000000000000, -1.200000000000, 40.123000000000, 1.200000000000, 0.000000000000] [] [13.0124, 300.3430, 300.3430, 1.2000, -1.2000, 14.0168, 1.2000, 0.0000] [-1.2000, 9.1886, 40.1230, 87.1284, 100.0200] [54.1768, 44.1648, 0.0000, 40.1230, 33.1479, 300.3430, 1.2000, 36.0520, 300.3430, 0.0000] [-1.2000, 69.0847, 40.1230] [100, 95, 1, 100, 60, 49, 300, 93] [300, 300, 40, 100, 1, 100, 40, -1] [] [0, 300, 40, 6, 43, 40, 60, 1, 6, 1] ["能进愉快看重", "?", "分公司航海路心理学明天找我南三环传真农行先这样万兴刚刚出去", "名称喜剧", "want would have will been all me think it got"] ["some", "do", "-"] ["-", "can how going did would my she going ok", "双核亲戚恢复不见月内以前", "get would hey why don't for okay we", "", "from", "--", "up going come mean that's and come one there", "?", "as"] ["when she in can't up did", "not", "隔壁"] ["a", "", "新天智能那也", "is time I'm going then here really the could", "?"] ["磁带", "-", "信海礼拜天影响"] ["", "", "", "--", "be", "say", "been get but", "to okay", "?", "back right look it's"] ["大棒", "--", "I'll", "", "if"] ["z", "x", "e", "u", "g", "x", "v", "p"] ["g", "b", "c", "t", "h"] ["f", "o", "g", "r", "m", "f", "p", "s"] ["t", "d", "j", "k", "e"] [2025-02-17, 2024-01-09, 2023-12-17] [2024-02-18, 2024-08-03, 2023-12-17, 9999-12-31, 2024-01-17] [2014-08-12, 2023-12-15, 2024-02-18, 2023-12-19, 2027-01-09, 2025-02-18, 2024-06-30, 2023-12-13, 2023-01-15, 2023-12-10] [9999-12-31, 2023-12-17, 2025-02-17] [] [2027-01-16 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00] [] [2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00, 2023-12-13 00:00:00] [2023-12-20 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00] [2024-06-30 12:01:02.123000, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30] [2024-01-09 00:00:00, 2023-12-13 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00, 2023-12-20 00:00:00] [] [2023-01-15 08:32:59.123123, 2008-09-11 00:00:00, 2023-12-10 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123] [2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123] [2024-01-08 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30] [2024-07-01 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00] +220 2147483647 -1795079002 936211637 2147483647 2025-06-18 2024-02-18 2014-08-12 2024-01-17 -- ? tell of get 词语你家才能赶快手机报表 美好哦啊你问火球谈判中天放风筝一份子国奥提货上机柜好好 ? 贰万五笔倒掉连成这次去暴力不低于大厦合兴 1 0 0.000000 1.200000 1.200000 40.123000 2025-02-17 00:00:00 2024-06-30 12:01:02.123000 2024-02-18 00:00:00 2014-08-12 00:00:00 2023-12-12 00:00:00 2023-12-11 00:00:00 2024-01-09 00:00:00 2024-01-19 00:00:00 [1, 0, 1, 1, 0] [0, 0, 0, 0, 0, 1, 0, 1] [0, 0, 0, 1, 0, 1, 1, 0, 0, 0] [] [26, -1, 9, 6, 0, 9, 1, 89] [117, -120, 0, 1, -128, -128, 0, -18, 1, 127] [1, 9, -1, -1, 1, -1, -121, 26] [-1, -1, 0, 27, 29, -1, -1, -34, 1, 1] [24641, -7488, -14160] [-28777, 31418, 32767, -12388, -29621] [6717, 0, 31494, 245, 22155, 20212, 32767, 1] [-21086, -29095, 4213, 24730, 0] [941733533, -457710571, 801976118, 2147483647, 1900505982, -426090430, 32679, 1736934862, -1812059770, 1510382647] [] [374948589, 311225243, 2147483647, -22179021, 2078092700, -1399537800, -1550990382, 1, -1285169276, 1542558370] [-2147483648, 147483648, 2147483647] \N [32679, -3465529564198593792, -1, 8772711460804720175, -9223372036854775808, -4612146539457591724, 9223372036854775807, 147483648] [1, 245, -9223372036854775808, 0, 147483648, -9223372036854775808, -9223372036854775808, 1, 147483648, 2516555236195612720] [] [5, -1, 0, -2, 1214341703] [] [] [-1, 0, 1, 0, 7, 1, 1, -1, 0, 2122837010] \N [92.053500000000, 22.152300000000, 0.000000000000, 50.047600000000, 40.123000000000, 82.082400000000, 100.020000000000, 0.000000000000, -1.200000000000, 0.000000000000] [1.200000000000, 1.200000000000, 300.343000000000, 100.020000000000, 40.123000000000] [300.343000000000, 0.000000000000, 300.343000000000] [300.3430, 70.1553, 0.0000, 40.1230, 45.0947, 98.1338, 0.0000, 0.0000] [300.3430, 40.1230, 300.3430, 4.0326, 100.0200, 57.1876, 22.0022, 0.0000, 82.0766, 23.0615] [0.0000, 79.0259, 100.0200, 0.0000, 100.0200, 0.0000, 40.1397, 40.1230] [300.3430, 85.0558, 40.1230] [1, 1, 71, 300, 100, 40, 100, 35] [76, 1, 40, -1, 40, -1, 26, -1] [] [62, 40, 100, 100, 300, 0, 40, 0, 40, 10] ["going", "-", "?", "宣传", "have", "no", "底价腾龙", "原因的儿孙们深爱老是什么的一改齐村小精灵公里智博"] ["was", "系诶邪恶", "--", "刀片服务器大河报平常改改安小洗衣服沐浴", "--", "of", "you the from me would but as", "then some have do with"] ["-", "know you hey be", "而是十二常用", "--", "out me just from", "-", "做好", "志强", "one see were as going no at", "事务所"] ["did", "位置", "see", "they", "", "?", "and out go a", "?", "短信", "--"] \N \N ["摄像机", "--", "-", "适合怎么办更新", "?"] ["广大下班层次寻找侠诺夜市王岩春上门", "--", "-", "多年", "here mean now time look your", "新年儿童换成笔记本", "", "some"] \N ["d", "q", "r", "m", "f", "p", "l", "j"] ["p", "l", "b", "c", "d"] ["u", "f", "g", "m", "v"] \N \N [2024-01-09, 2024-06-30, 2027-01-09] [2024-01-31, 2023-12-19, 2025-06-18, 2025-02-17, 2025-02-17, 2023-12-15, 2023-12-15, 2023-12-17, 2023-12-15, 2025-06-18] [2024-02-18 00:00:00, 2024-01-19 00:00:00, 2024-06-30 12:01:02, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2024-06-30 12:01:02, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00] \N [] [2005-08-19 00:00:00, 2025-02-17 00:00:00, 2024-01-08 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00] [2023-12-18 00:00:00, 2023-12-19 00:00:00, 2023-12-16 00:00:00] \N [9999-12-31 23:59:59, 2023-12-11 00:00:00, 2027-01-09 00:00:00] [2024-01-08 00:00:00, 2025-02-18 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] [2024-01-09 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2024-01-17 00:00:00, 2025-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00] [2026-02-18 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59] [2023-12-16 00:00:00, 2023-12-09 00:00:00, 2023-12-15 00:00:00, 2023-12-10 00:00:00, 2004-04-20 00:00:00, 2024-02-18 00:00:00, 2001-01-02 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00] [2019-01-07 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2023-12-20 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00] +221 1853809208 0 197583135 147483648 2024-07-01 2027-01-09 2023-12-15 2023-12-09 \N ? 身份 n 传真网上多些 - -- - \N 0 0.000000 9.003300 100.020000 -1.200000 2023-12-12 00:00:00 2027-01-16 00:00:00 2025-06-18 00:00:00 2025-02-18 00:00:00 9999-12-31 00:00:00 9999-12-31 23:59:59 2024-01-19 00:00:00 2023-12-14 00:00:00 [0, 0, 1, 1, 1, 0, 0, 1] [1, 1, 0, 1, 0, 0, 1, 1, 0, 1] [0, 1, 1, 1, 1] [0, 1, 1, 1, 0, 0, 1, 1] [0, -1, 127, 127, -128, 127, 0, -16] [46, -128, -128, 127, 46] [-69, -74, 127, 69, 8, -1, -24, -110, -128, 0] [127, 117, -66, 0, 1] [1, 32767, 1, -26982, 1, -32768, 0, -27748, 1, 5967] [-4783, 1, -4337, -1215, 1, 14610, 1, -32768] [-1, 0, 15838, 5038, 13031, 245, 4977, 245] [] [1, 147483648, 706182894, -2040175762, 387512709, 599409680, 217169330, 1286438986, 147483648, 32679] [2147483647, 0, 32679, 1076464216, 32679, 147483648, -2147483648, 147483648, -1, 576827297] [-1704695459, 1310451267, 1, -258168916, 0, -1534780876, 32679, 147483648, 32679, 0] [644871370, -1180507410, -1, 293598379, 1849901679] [1, 153893, 147483648, 32679, -4644137190041202691, 245, 245, 32679, 6135419226676963812, 32679] \N [-3373057, -9223372036854775808, 147483648] [32679, 710776039, 7126921435094189525, 147483648, 9223372036854775807] [-1, 4, 0] [1, 0, 5, -1746069199, 1164573938, 648740820, 0, -1701567707] [-1, 0, -1, -2, 3, 7, 8, 0, 0, -2] [-840791855, 1, 1127570936] [-1.200000000000, 11.167300000000, 300.343000000000] [300.343000000000, 1.200000000000, -1.200000000000] [-1.200000000000, 91.118300000000, -1.200000000000, 40.123000000000, 0.000000000000] [] [47.1431, 300.3430, 100.0200, 82.1808, 45.1039, -1.2000, 1.2000, 5.0102, 300.3430, 98.0660] [93.1269, 1.2000, 40.1230, 300.3430, 0.0000] [] [-1.2000, 100.0200, 100.0200, 0.0000, 80.1081] [55, 51, 40, -1, 51, 1, 1, -1, 1, 300] [1, 78, 94, 40, 300, 56, 9, 40] [] [100, -1, 0, 40, 13, -1, 100, 0, 58, 1] [] ["代玉", "I'll come see", "-", "私人大键盘添加大忙人同志们粘贴冀海潮公章代表一直吃饭哪", "a", "you're", "about", "--"] ["?", "协议", "女朋友机型用眼研究院塔式问题志光还在研普", "your", "with", "?", "面对", ""] [] ["I'll well or because she you and how be", "老了容量前台", "say your one your out my a", "e", "can't", "we", "x", "相隔"] ["can not up was you're hey you", "", "", "王青优于高高兴兴勾丽娜屏蔽机关可笑安排好小雨吃住", "-", "-", "q", "in not I'll well it's"] ["--", "年后王青家庭套多初八", "--", "零捌招工共享入职最美丽原则面包", "?", "--", "政府李峰李景林稍后外聘先这样存续验货单用于一想", "j"] ["x", "--", ""] ["p", "l", "z", "r", "o", "i", "o", "r"] ["m", "c", "c", "z", "x", "s", "y", "k"] ["o", "b", "k", "y", "a"] ["l", "a", "n"] [2026-02-18, 2024-01-31, 2024-02-18] [2024-07-01, 2023-12-16, 2024-01-08, 2023-01-15, 2026-02-18] [] [2024-01-19, 2024-08-03, 2023-12-15, 2023-12-16, 2023-12-18, 2025-06-18, 2024-06-30, 2026-01-18, 2023-12-12, 2024-02-18] [2024-02-18 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2023-12-17 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00] [] [2023-12-19 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30] [2025-06-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00] [2025-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2024-01-09 00:00:00] [2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00] [2023-12-14 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00, 2002-12-12 00:00:00, 2023-12-11 00:00:00] \N [2023-12-14 00:00:00, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00] [2023-12-20 00:00:00, 2023-12-16 00:00:00, 9999-12-31 23:59:59, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00] +222 1 1159449475 1 2147483647 2024-01-08 2025-06-18 2023-12-16 2027-01-16 -- in why would can't you're that's good at is \N didn't - -- as 1 1 70.095800 11.156800 100.020000 43.064900 9999-12-31 00:00:00 2024-01-08 00:00:00 2023-12-20 00:00:00 2023-12-18 00:00:00 2024-08-03 13:08:30 2027-01-16 00:00:00 2024-02-18 00:00:00 2024-07-01 00:00:00 [0, 1, 0, 0, 1, 1, 0, 0, 1, 1] [1, 0, 0, 1, 0] [1, 1, 0, 1, 1] [] \N [1, -39, -1, 79, -128, 73, 5, -128] [-128, -1, -99, 113, 95, 2, 119, 0] [] [] [245, 32767, 11085, 1, 245] [] [30151, 32767, -1, 245, 32767, -27900, -12287, 28971, -27882, 28250] [0, 1590805318, 32679] [1, 0, 2147483647] [917596047, 147483648, 1054039275, 0, -2147483648, -1, -1052202486, 1] [-1868344648, 1, 2098051730, -909929905, 2147483647, 1553468596, 2147483647, 0, -235362001, 1281020239] [-1976048, -120337279498382431, 245, 5827312419054617375, 1320699897, -9223372036854775808, -7102512773590325125, -4582183315854224127, 245, 245] [6132597333564044684, -9223372036854775808, 5411614, 1, 245, 8001573454422628984, -8597592334412990232, 245, -2360849, 1946388978101735568] [32679, -1, 245, -9223372036854775808, -1, 32679, 147483648, -5986636349025441546, 6577898533049742326, 9223372036854775807] [] [1, -2, -1, -1, 2, 1, 1, 4, -1, -2] [1, 0, 1] [-1, 3, 1, -2, 1] [] [40.123000000000, -1.200000000000, 36.023500000000] \N [300.343000000000, 91.075400000000, 22.134700000000, 100.020000000000, 100.020000000000] [] [12.1049, 91.1266, 0.0000, 100.0200, 34.0207, 1.2000, 23.0502, 45.1575] [] [42.1841, 100.0200, 46.0160, 1.2000, 15.0158, 1.2000, 44.1574, 1.2000] [] [] [0, 40, -1, 100, 17, 100, 100, 1] [300, 100, 30, 100, 32] [1, 59, 40, 52, 100, 0, -1, 72, -1, 29] ["不便", "", "?"] ["饱和", "-", "老了中创春子在线乙方吵架前台热备软件个哦", "his", "u"] ["", "刀片", "嘉运达"] ["something", "慢慢", "-", "", "something because to", "could a or don't they were mean tell", "now he's say get now here if really something would", "看法视讯不知道王星我不会玩", "大公司", "郭晓"] \N [] [] ["out", "图片出去看来", "?", "来了", "-"] ["u", "v", "u", "w", "w", "f", "c", "k"] ["f", "z", "v", "w", "j", "d", "r", "d", "h", "d"] ["h", "q", "w"] ["b", "p", "u"] [2024-01-19, 2024-07-01, 2025-02-18] [2024-07-01, 2024-08-03, 2025-06-18, 2023-12-12, 2023-12-09, 9999-12-31, 2025-02-17, 2024-07-01] [2024-08-03, 2023-12-20, 2024-02-18] [2014-03-14, 2024-02-18, 2023-12-10, 2023-12-13, 2023-12-18] [2023-12-19 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2005-11-13 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00, 2023-12-19 00:00:00] [2023-12-17 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2024-01-19 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59] [] [2023-12-10 00:00:00, 2023-01-15 08:32:59.123000, 2002-12-21 00:00:00, 2024-08-03 13:08:30, 2024-07-01 00:00:00] [2023-12-19 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-13 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2023-12-11 00:00:00] [2023-12-18 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2023-12-10 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59] [] [2023-12-14 00:00:00, 9999-12-31 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2023-12-15 00:00:00, 2024-01-17 00:00:00] [] +223 -2147483648 -1159661661 1 -2147483648 2025-02-18 2023-01-15 2025-02-17 2024-01-17 飞达 改动 I'll why was 很细那个办法会话电子提出也是正式版误区原来信天游 mean 独显 \N 0 300.343000 0.000000 61.169800 300.343000 2023-12-20 00:00:00 2023-12-12 00:00:00 2023-12-14 00:00:00 9999-12-31 00:00:00 2023-01-15 08:32:59.123123 2026-02-18 00:00:00 2025-02-17 00:00:00 2024-02-18 00:00:00 \N [1, 0, 0, 0, 0] [0, 1, 0] [1, 0, 0, 1, 1, 0, 1, 0, 0, 1] [116, 0, 1] [-106, 24, 103] [-1, 92, 1] [0, -19, -128, -24, 127, -88, -40, -1] \N [] [-1, 32767, 9709] [13673, -17537, 18853, -11978, -23427, 30654, 21931, 32767, 337, 245] [1, -1532822950, 1, 1301017401, -2147483648, 147483648, 1677216617, -1893275689] [-714317427, 1, 1, -2147483648, 53786529] [-2147483648, 1, -1728728863, -1916431132, -1] [] [32679, 1804497890, -4525584575368136266] [-7692026, 245, 32679, 32679, -9223372036854775808, 4461112497832393113, 147483648, 9223372036854775807, 8060868288222392841, -9223372036854775808] [9223372036854775807, 1, 9223372036854775807, 32679, 9223372036854775807] [-572176839, 1909891446, 1, -2136159, 1, 32679, 1, 245] [] [-1, 195849128, 1, 0, 5] [0, 0, -1, -1, -1, -2, 1838621447, 0] [-1577217816, 0, 7, 0, 1, -1, 1, -1] [300.343000000000, 300.343000000000, 23.154900000000, 0.000000000000, 85.025800000000, 40.123000000000, 1.200000000000, 85.052400000000] [1.200000000000, 18.190600000000, 300.343000000000, 0.000000000000, 0.000000000000] [38.079500000000, 37.149100000000, 100.020000000000, 8.022900000000, 100.020000000000] [300.343000000000, -1.200000000000, 100.020000000000, 0.000000000000, 46.111100000000, 37.086600000000, 100.020000000000, 40.123000000000, 100.020000000000, 1.017700000000] [-1.2000, 1.2000, -1.2000] [300.3430, 100.0200, -1.2000, 11.1906, 2.1468] [300.3430, 1.2000, 100.0200, 1.2000, 300.3430] [53.1147, -1.2000, 1.2000, 45.1972, -1.2000] [0, 100, 100, 300, 100, 92, 100, -1] [75, 56, 58, -1, 40] [300, 0, -1] [] ["up", "", "模拟", "普通", "陌生人", "用眼", "really", "?", "the", "-"] ["询问过用管早晨数量天马经销商真正以以上的锐康收吧聚兴", "", "-"] ["回公司回家大兵新增电厂沃尔普最近是吗阶段", "-", "j", "", "all"] ["well", "you", "b"] ["?", "--", "this", "", "come and from could see was it out"] ["骗人", "规定主机中小企业分清这种小曹有车吗年底知道了", "was", "like they as want some it's been have", "want", "", "所作", "b"] ["", "实现", "or"] [] ["s", "a", "t", "n", "p", "o", "b", "f"] ["a", "z", "n", "k", "i", "a", "b", "c"] ["z", "u", "k", "c", "x", "l", "d", "w"] ["b", "t", "c", "s", "f"] \N [2023-12-18, 2023-12-15, 2025-02-17, 2023-12-16, 2024-06-30, 2023-12-15, 2023-12-18, 2023-12-15] [2023-12-11, 2024-01-31, 9999-12-31, 2027-01-09, 2024-02-18, 2023-12-15, 2023-12-17, 9999-12-31] [2025-06-18, 2023-12-16, 2023-12-10] [2023-12-17 00:00:00, 2024-07-01 00:00:00, 2027-01-16 00:00:00] [2024-07-01 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2024-01-09 00:00:00, 2027-01-09 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00] [2027-01-09 00:00:00, 2014-08-12 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00] [2027-01-09 00:00:00, 2023-12-12 00:00:00, 2018-12-17 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 2026-02-18 00:00:00] [2024-01-19 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2023-12-13 00:00:00] [2025-02-18 00:00:00, 2014-08-12 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2025-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2024-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00] [2024-08-03 13:08:30, 2024-01-17 00:00:00, 2024-06-30 12:01:02.123000, 2014-08-12 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30] [] +224 2147483647 1931337541 840107393 929841703 2024-02-18 2023-12-11 2025-06-18 2025-02-18 is 煤炭 同力那几天津磁盘以下很难 蝴蝶 ? like up that's there have something what it's ? 上门南三环 \N 0 300.343000 29.034100 75.092900 1.200000 2024-01-09 00:00:00 2025-06-18 00:00:00 2024-02-18 00:00:00 2023-01-15 08:32:59.123000 2023-12-11 00:00:00 2023-12-11 00:00:00 2023-12-13 00:00:00 2023-12-13 00:00:00 [0, 1, 0, 0, 0, 0, 0, 1, 1, 1] [1, 0, 0, 1, 0, 1, 1, 1] [0, 0, 1, 1, 0, 0, 0, 0] [1, 1, 1] [102, 0, 122, 1, 1] [1, 127, 1, -1, -128, -22, 127, -128, -128, -1] [-1, 1, -31, 1, -1, -80, -21, -1] [-128, -1, -128, 3, -1, 52, -128, 127, 1, 0] [29723, 12377, -8562, 17056, 245, 0, 245, 0] [14229, 32767, 32767, -1, 245] [-10462, 32767, -3986, 0, 23573, -32768, 27606, 15935, 1, -9008] [22568, -29059, -30721, 16197, -13174, 245, -1, 245] [32679, -1196782280, 276539631] [105656089, -12658456, 681260825, 32679, 147483648, 0, 887393860, -215721026, -1762152566, 147483648] [32679, -2147483648, 1, 0, -1253387741, -2147483648, 32679, -1, -1, -246107188] [849310836, 147483648, 1165079693] [147483648, -6658343214318848374, 9223372036854775807, 5213723699460134364, 245, 3013148151641638610, -1, 7167359, -5941189846283326470, 9223372036854775807] [4803052692900965898, 0, -2853944798861294084, -5638194446253894678, -7254541846859323547, 0, 245, 1] [1, 147483648, -1, 245, -1, 245, -1, 5805216039902098248, -7644068147957125134, -1742139] [245, -2971356390597983472, -1679266757651768312] [8, 0, -2] [-2, 1, 5, 0, 8, 0, 1538655212, 2] [0, -2, 1, -2, 8] [-2, -1683318481, 1, 0, 0, 0, 1, 4, 0, 0] [-1.200000000000, 47.160600000000, 28.124300000000] [0.000000000000, -1.200000000000, 1.200000000000] [] [4.048800000000, 100.020000000000, 0.000000000000, 0.000000000000, 1.200000000000, 100.020000000000, -1.200000000000, 4.191800000000, -1.200000000000, 300.343000000000] [86.0849, 30.1838, -1.2000, 75.1529, 92.0677, 46.1557, -1.2000, 0.0000] \N [0.0000, 40.1230, 40.0521, 100.0200, 300.3430, 77.0815, 40.1230, -1.2000] [-1.2000, -1.2000, 40.1230, 100.0200, 0.0000] [59, 100, 300, 0, 300, -1, 24, 39, -1, 1] [300, 100, 300, -1, 1, 2, 51, 36] [] [] ["were that", "could", "was", "王岩春", "--"] ["want", "正在朱晓明才是一千多前辈商都都还慧眼", "me would time why about"] ["--", "?", "would about", "清单", "", "不小", "--", "I'm"] ["", "", "okay"] ["?", "主演寸草心留个条款拉美老大爸妈李金才奖励", "", "", "right one it's", "查证智者通话零度祝福推磨请问", "his go then to ok up now did know", "--", "", "for this don't did you're there"] [] ["-", "早点", "词条", "海花", "i", "大品牌弱阳性表示", "up it's ok got", "at for of as you're"] ["刘畅培训大忙人", "职务", "a", "电影意思肯定宏泰提到天海联想", "雪城热备策略正式版那六好友", "--", "?", "?", "-", "河南经销桌子很早柒服装恢复"] ["h", "a", "b", "z", "b"] ["y", "d", "v", "r", "r"] ["p", "n", "m"] ["b", "m", "j", "i", "g"] \N [9999-12-31, 2024-07-01, 2024-08-03] [2023-12-12, 2025-02-17, 2024-01-09] [2025-02-18, 2026-02-18, 2024-02-18, 2023-12-11, 2025-02-17, 2019-02-08, 2025-06-18, 2024-02-18, 2023-12-16, 2024-02-18] [2023-12-20 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00] [2025-02-17 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2023-12-16 00:00:00, 2023-12-16 00:00:00] [] [2024-06-30 12:01:02, 2023-01-15 08:32:59, 2026-02-18 00:00:00] [2025-02-18 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00] [2024-01-08 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2023-12-16 00:00:00] [2024-08-03 13:08:30, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00] [2027-01-09 00:00:00, 2027-01-16 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00] [2023-12-13 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00] [2026-01-18 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59] [2024-01-08 00:00:00, 2027-01-09 00:00:00, 2025-06-18 00:00:00] [2023-12-14 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-17 00:00:00] +225 0 -2064039616 253129953 894365116 2023-12-13 2025-06-18 2011-10-21 2023-12-15 n 运行教练扬子授权开通舍得品放用得上先锋 \N 在欧普 ok hey ? go 1 1 38.055100 40.123000 40.123000 30.194400 2024-02-18 00:00:00 9999-12-31 00:00:00 2023-12-13 00:00:00 2024-01-19 00:00:00 2023-01-15 08:32:59.123123 2026-01-18 00:00:00 2025-06-18 00:00:00 2025-02-18 00:00:00 [1, 1, 0, 0, 1, 1, 1, 1, 0, 1] [0, 1, 0] [1, 1, 0, 1, 0, 1, 1, 0, 0, 0] [0, 1, 1] [92, 103, -128] [2, 0, 3, -1, 3, 90, 0, -128] [9, 1, 1, 0, -128] [0, 2, -11, 1, 127] [-26627, 0, -29121, 1940, 0, 32767, -32768, 0, -24424, 0] \N [] [] [147483648, -2147483648, -2105418221, 1908352473, -1, 0, 0, 1891691328] [] [0, 32679, 915092096, 2147483647, -2147483648, 1494797097, 147483648, -1158711269, 0, -2147483648] [] \N [-9223372036854775808, 0, 1, 147483648, -1] [32679, -1, -351501934, 9223372036854775807, 245, 147483648, 32679, -5730966914399321440] [6947380, 1, 1, 1, 9223372036854775807, 245, 1, 245, -1, 9223372036854775807] [-2, 1, 8, -1, -2, 0, 9, -904240678] [0, 1, -2, 1, 3] [0, -1, 1, 254199159, 0, 1566333215, -778484421, 0] [8, 0, 2, -1, 2] [40.123000000000, 40.123000000000, 4.020100000000, 300.343000000000, 99.055400000000, 300.343000000000, 1.200000000000, 40.123000000000, 100.020000000000, 41.177000000000] [] [40.123000000000, -1.200000000000, 0.000000000000, 40.123000000000, 0.000000000000] [] [-1.2000, 300.3430, -1.2000, 300.3430, 0.0000, 40.1230, 19.0112, 99.1463] [1.2000, -1.2000, 0.0000, 0.0000, 300.3430, 12.0847, 40.1230, 100.0200, 19.1037, 40.1230] [100.0200, 0.0000, 0.0000, 18.1692, -1.2000, 300.3430, 0.0000, 33.0006] [48.0927, 300.3430, 1.2000, 0.0000, 300.3430, 0.0912, 49.1205, 86.0560] [] [100, 1, 37, 40, 300, 36, 300, -1, 100, -1] [100, 46, 300, 19, 40, 40, 1, 100] [300, 40, 300, 1, 70, -1, 300, 100, 98, 42] ["收尾兼容性", "time really out want", "-", "about here mean here or did", "--", "", "been", "i not how", "good if and now in him can't", "字库文件"] ["he", "c", "c", "q", "", "--", "老同学不问乐凯下一部分那一片集团结案报告漫长", "all", "my oh something I'm mean that's been my I'm go", "who yes from as good there right from"] ["--", "就爱个星月应该", "don't out well mean out how back did her", "going", "-"] ["--", "其中欧回来对方总之不要错子夜濑尿虾", "?", "", "o", "这上面谁在妈的看法网站壹仟若虚", "he's", "yeah as do just what", "that's", "?"] \N ["--", "是你呀", "", "who", "-", "?", "投缘心里的双核通信旅之星鑫创你杂能进今晚", "were", "good there I'll as it ok not as", "that"] ["", "", "不忙", "g", "", "", "on", "--", "思科", "a"] ["色戒柏霖订货后勤教练脱机核算参加", "we in like had this going", "?", "", "and he's I'm or but she something"] \N ["i", "z", "a", "g", "s", "f", "c", "u"] ["q", "q", "g", "s", "w", "a", "k", "s", "f", "y"] ["x", "g", "f", "h", "i", "r", "m", "g", "b", "h"] [2023-12-15, 2027-01-16, 2025-02-18, 2023-12-16, 2023-12-14, 2024-01-17, 2023-12-19, 2024-01-08] [2024-07-01, 2023-12-09, 2024-01-09, 2024-01-31, 2024-02-18] [2023-01-15, 2025-02-18, 2023-12-16, 2024-08-03, 2025-06-18, 2024-02-18, 2025-06-18, 2024-07-01] [2024-07-01, 2023-12-17, 2024-07-01, 2023-12-19, 2024-07-01, 2024-06-30, 2024-01-08, 2025-02-18, 2024-08-03, 2027-01-16] [2023-12-14 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2024-08-03 13:08:30, 9999-12-31 00:00:00, 2024-06-30 12:01:02, 2023-12-16 00:00:00] [2024-02-18 00:00:00, 2023-12-19 00:00:00, 2024-01-08 00:00:00] [2024-01-31 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00, 2023-12-13 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2023-12-13 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-08 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2024-01-17 00:00:00] [2023-12-20 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00] [2023-12-17 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2018-02-09 00:00:00, 2026-02-18 00:00:00] [2003-05-15 00:00:00, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2023-01-15 08:32:59.123123] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30] [9999-12-31 23:59:59, 2023-12-14 00:00:00, 2024-06-30 12:01:02.123000] +226 1812029442 136136247 1 2147483647 2025-02-17 2023-12-20 2023-12-09 2023-12-19 ? 零度招生天津光电果断 with going mean look 青春 ? \N 0 0.000000 1.200000 100.020000 70.158500 2023-12-18 00:00:00 2026-01-18 00:00:00 2024-07-01 00:00:00 2008-07-04 00:00:00 9999-12-31 23:59:59 2025-02-18 00:00:00 9999-12-31 23:59:59 2023-12-19 00:00:00 [1, 1, 1] [1, 0, 1, 0, 0, 1, 1, 1] [0, 0, 0] [0, 0, 0, 0, 1] [] [0, 4, -81, 0, 8, 16, 2, 1, 7, -128] [127, 1, 94, -1, -128] [-1, 127, -125, -1, -1, 1, -128, 9] [21215, -17974, -5367, 19951, -29001] [-6697, -1150, -8968, 0, -1] [245, -6602, 32767] [30505, 32767, -32768, 1, 0] [-1, -863712092, 1, -411821798, -1101829931, 2147483647, 32679, 1213794649, 1122357304, 1] [1, 2147483647, 0, 32679, 1731006317] [] [] [245, 147483648, -683086, -1292734, -2410254956236911431, 245, -1, -9223372036854775808, -9223372036854775808, -597048858630912359] [6726237634734933548, 9223372036854775807, 1, 1, 2162101111297284160, -1, 1, 32679, 1, -7297428] [] [] [-1547967047, -558853732, -1, -1, 1] [] [0, -279041995, 1, 1, 1457631356, 2, -2, 2] [-2, 0, -1616030500] [] [-1.200000000000, 100.020000000000, 0.000000000000, 97.092500000000, 30.086400000000, 100.020000000000, -1.200000000000, 0.049700000000, 1.200000000000, 52.054100000000] [56.191600000000, 300.343000000000, 0.000000000000] [] [91.1462, 0.0000, 100.0200, 300.3430, 78.1555, 66.1390, 40.1230, 300.3430] [1.2000, 300.3430, -1.2000, 84.0948, 40.1230, -1.2000, -1.2000, 0.0000, 40.1230, 300.3430] [0.0000, 300.3430, 22.0490] [40.1230, 1.2000, 40.1230, 1.2000, 1.2000] [100, 300, 23, -1, 0] [-1, 45, 1, -1, 73, 3, 0, 1] [300, 67, 1, 100, -1, 300, 88, 40, 100, 48] [0, 21, 100, 100, 56, 77, 0, 52] ["记得", "也要", "do because as it's could to how or at", "手里华林弟子送过去放弃日月丰写吧孔艳资料大品牌", "a been want did when back her that", "p", "资质", "长城属于丹尼斯他妈等到瑞科人员名单承担才好数字显示", "承诺年底规格是你或有事情机关接触最新", "-"] ["", "三阳开泰惠达多家", "代码广场电视台", "o", "哪呢个舒展真实需求天下无贼键盘孔艳名字谢谢说不"] [] [] ["打印机", "that", "捕鲸船投资简单给我吧党经理忘不了是你呀身份证", "?", "先锋增值看到"] [] [] [] ["r", "m", "o", "h", "y", "k", "s", "s", "v", "c"] [] ["j", "l", "j"] ["b", "u", "v", "y", "b"] [2024-01-19, 2023-12-15, 2024-01-19, 2026-02-18, 2024-01-31] [2024-07-01, 2025-06-18, 2024-08-03, 2023-12-09, 9999-12-31] [2023-12-20, 2025-02-17, 2023-12-09] [2024-06-30, 2025-02-18, 2023-12-19, 9999-12-31, 2026-02-18, 2027-01-16, 2023-12-14, 2025-02-18, 2023-12-14, 2023-12-09] [2023-12-17 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2000-02-22 00:00:00, 2019-01-10 00:00:00, 2027-01-09 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00] [2023-12-18 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2018-04-28 00:00:00, 2025-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-10 00:00:00, 2023-12-15 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00, 2023-01-15 08:32:59, 2007-06-18 00:00:00] [2024-08-03 13:08:30, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2024-06-30 12:01:02, 9999-12-31 00:00:00, 2023-12-18 00:00:00] \N [2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-01-08 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00] [2024-01-31 00:00:00, 2023-12-16 00:00:00, 2024-01-09 00:00:00] [2024-01-09 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-19 00:00:00, 2023-12-19 00:00:00] [] [2024-01-08 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00] [2025-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00, 2005-11-28 00:00:00] [2023-12-15 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-10 00:00:00, 2024-01-19 00:00:00, 2026-01-18 00:00:00] +227 942842027 -679051610 1030344901 32679 2023-12-14 2023-12-10 2024-01-19 2027-01-09 多多邮寄有意向小灵通一起就像大公司 接收 think 乔总 ? don't 热销底价 1 1 40.123000 1.200000 0.000000 1.200000 2023-12-15 00:00:00 2024-02-18 00:00:00 2023-12-09 00:00:00 2023-12-15 00:00:00 2023-12-09 00:00:00 2023-12-17 00:00:00 2023-12-09 00:00:00 2027-01-09 00:00:00 [] [1, 1, 0, 1, 0, 1, 0, 0, 0, 1] [0, 1, 1] [1, 0, 1, 0, 1] [] [-115, -82, -61] [-104, 127, -128, -1, 65, 106, 0, -1, -1, -128] [127, 127, 127, -3, -47, 80, 0, -128, 89, 8] [0, 245, 0] [-32768, 32767, -1] [0, 245, 245, 13404, 32767, 245, -29764, -1, 16276, 1] [-30280, 27858, 245] [0, -2031831773, -2147483648, -2147483648, -1481020006, -696431563, 420789872, 1442040806, -542621247, 539020832] [-1, -2147483648, -759163334, -1, -1, -1, -1329175098, 1607687498, -1161651170, 0] [-1, -1051631380, 0, -1, 32679, 1, -1295868814, -1859480202, 2147483647, 1572645115] [0, 1, 727576267, 32679, 37406483] [-4981444, 9223372036854775807, 32679, 9152741294094769387, 1567607, -1, 3549586395701895570, 4420380341385379480] \N [-9223372036854775808, 32679, 1, 147483648, -9223372036854775808, 5028339317561309327, 0, 0] [1, -7650047681508593265, -9223372036854775808, -9089733107383534898, -2799250598368998912, -6431981, -7545728, -943176272, 9223372036854775807, 6774715993397624965] [1, 1, -2] [8, 1, 1] [] [835564286, -705236881, -1130031611] [72.167300000000, 1.200000000000, 2.157600000000] [100.020000000000, 100.020000000000, 52.195800000000, 0.000000000000, 300.343000000000] [100.020000000000, 1.200000000000, 14.107700000000, 100.020000000000, 100.020000000000, -1.200000000000, 68.152900000000, -1.200000000000, -1.200000000000, 59.086300000000] [22.027000000000, -1.200000000000, 100.020000000000, 1.200000000000, -1.200000000000, 46.194200000000, 31.013900000000, 40.123000000000] [20.1096, 1.2000, 1.2000, 40.0403, 40.1230, 1.2000, 38.0458, 100.0200, 40.1230, 1.2000] \N [-1.2000, 52.0358, -1.2000] [40.1230, 100.1487, 0.0000, 100.0200, 300.3430] [89, 100, 96] [100, -1, 1] [48, 55, 0, 300, 40, 300, 0, 48, 40, 300] [100, 100, -1] ["are", "?", "黄经理", "-", "惭愧", "it's", "q", "内外", "get", "徘徊生物华林通过单单熟悉企业管理拿住老总"] ["x", "had right how oh they think will all be", "?", "think", "有什么服装确认现在", "夜间", "?", "多点"] ["to", "王总签不签都", "how how as", "同事历来代理价格出团天数没沈晓海全名任性", "诺顿", "ok I'll", "were they well", "be"] ["", "there", "?", "", "how my could to got as one some all so", "at", "呀库存全部", "停留"] ["新乡市", "?", "are well", "something", ""] ["-", "a", "水晶福利我能至高能量", "?", "f", "yes how in", "模拟深夜跟我想过道理", "千金"] ["?", "-", "hey", "-", "about do", "say", "红包", "偶然", "寸草心三种显卡调价高级总代理联盟", ""] ["a", "--", "did from about because at her you or hey"] ["f", "s", "q", "l", "a"] ["c", "d", "n"] [] ["o", "f", "i"] \N [9999-12-31, 2026-01-18, 2023-12-17, 2027-01-09, 2026-01-18] [] [] [2023-12-11 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00] [2024-07-01 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2024-01-09 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00] [2023-12-19 00:00:00, 2023-12-13 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00] \N [2024-01-17 00:00:00, 2023-12-16 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000] [] [] [] [2023-12-16 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2014-08-12 00:00:00] [] [2027-01-09 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00] +228 -1 0 1 32679 2023-12-17 2024-07-01 2026-01-18 2024-01-19 \N -- -- 只要 首位过节费 -- when - \N 1 43.139100 \N 300.343000 40.123000 2023-12-13 00:00:00 2024-01-17 00:00:00 2023-12-10 00:00:00 2023-01-15 08:32:59.123000 2023-12-12 00:00:00 2006-08-11 00:00:00 2025-02-18 00:00:00 9999-12-31 23:59:59 [0, 0, 1, 0, 0, 1, 1, 0] [0, 0, 1, 1, 0, 0, 1, 0, 0, 0] [1, 1, 0, 0, 1, 0, 0, 0, 1, 1] [] [-44, -113, 8, -1, 1] [-112, -76, -128] [94, -1, 1, 1, 1, -60, 0, -1] [4, 1, 1] [18946, -1, 10321, 245, -29998, 1, -19034, 245, 21207, -1] [27158, -1, 245, 245, -3737] [-16725, 32767, -32768] [] [0, -1, 32679] [] [] [] [-9223372036854775808, 0, -1, 32679, 9223372036854775807, 6254870742911608956, 147483648, 32679, 32679, -2759279798838111798] [-1, -9223372036854775808, 6162128] [-1385639124075001354, -5808831041215578592, 32679, -9223372036854775808, -549947542] [] [-2, -1, 9] [-1, 6, 1, 22146071, 0, -1, -2, -2, 0, -2] [4, 6, 0] [-1, -2, 942388317, 3, 2] [100.020000000000, 40.123000000000, -1.200000000000, 300.343000000000, 300.343000000000, 1.200000000000, -1.200000000000, 85.126700000000] [20.101200000000, 53.080600000000, 300.343000000000] [1.200000000000, 55.118700000000, 300.343000000000, 31.077200000000, 40.123000000000] [40.123000000000, 59.073200000000, 0.000000000000, 65.011800000000, 40.123000000000, 1.200000000000, 99.040200000000, 1.200000000000, 1.200000000000, 1.200000000000] [100.0200, 40.1230, -1.2000] \N [-1.2000, 0.0000, 1.1251, 300.3430, 100.0200, 0.0000, 0.0000, 0.0000] [23.1040, 0.0000, 89.0073, 1.2000, 40.1230, 0.0000, 300.3430, 1.2000] [20, -1, -1, 40, 0, 1, 0, 40] [1, 1, 100, 300, 300] [100, -1, 90] [26, 1, 0, 300, 0, 300, 100, 46, 1, 0] ["?", "some good", "who that want go or was", "", "日期", "his", "e", "往年新国腾讯发放是你或只要"] ["工程签订加点加为一套我找同学专业你不", "he's", "z", "?", "兄弟视听", "", "is", "静静", "--", "h"] [] ["--", "s", "一套林慧耽误五洲那段文件", "各位", "双机备份"] ["h", "?", "l"] ["time", "-", "张姐", "", "--", "I'll", "长得不再适合伤感弟子姑娘通过热备没错", "g"] ["--", "p", "-", "of", "--"] ["I'll", "瑞科", "-", "so", "--", "", "架子盘点键鼠张峰灵生瀚海聂柯占用难过", "见见知心", "注册表", "to"] ["v", "m", "b", "f", "n", "o", "e", "t", "x", "c"] \N [] ["v", "e", "q", "k", "d", "f", "c", "j", "t", "p"] [2023-12-17, 2024-01-19, 2024-01-08, 2024-07-01, 2027-01-09] [2027-01-09, 2005-08-22, 2025-02-18, 9999-12-31, 2025-02-17, 2014-08-12, 2026-02-18, 2025-02-17] [] [] [2014-08-12 00:00:00, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [2023-12-13 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2024-07-01 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00] [2023-12-16 00:00:00, 2023-12-13 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00] [9999-12-31 23:59:59, 2023-12-16 00:00:00, 2025-02-18 00:00:00, 2023-12-10 00:00:00, 2002-07-21 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2017-02-04 00:00:00] [2024-01-17 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59.123000] [2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00, 2027-01-09 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00] [2024-08-03 13:08:30, 2025-02-18 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00] [2012-03-09 00:00:00, 2014-08-12 00:00:00, 2011-09-12 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [2026-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [] [2024-01-08 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00] +229 390329028 1586013335 -2147483648 244518358 2026-02-18 2024-02-18 2024-07-01 2027-01-09 ? 词典 退信你用手机吧主做飞信总价 欧洲哭泣严重相隔不起说不 do 排列哈皮许文供货学习 -- k 0 1 53.012300 \N 82.003500 1.200000 2023-12-09 00:00:00 2023-12-10 00:00:00 2024-02-18 00:00:00 2023-12-14 00:00:00 2014-08-12 00:00:00 2026-02-18 00:00:00 2023-12-11 00:00:00 2023-12-12 00:00:00 [0, 1, 0, 0, 1, 1, 0, 1] \N [0, 0, 0, 0, 1, 0, 1, 1, 0, 0] [0, 0, 1, 0, 0] [] [-1, 1, 2, 9, 0, 127, -102, 4] [8, 0, 127, 0, 0, -1, -81, 0, 0, 1] [] [32767, 245, -19551, 24094, 28843, -32768, 15559, -3628] [] [5769, 25881, -13483, -32768, 2490, 1, 15232, 6175] [3189, -20099, 0, 26054, -32488, -1, 245, 12663, 245, 1] [622466729, -2147483648, 1656078334, -1823998586, 1952459663, -1645382336, 1966736962, 2147483647, 2059631355, 147483648] [-1757904446, -529053928, 147483648, -1, -1, 646948661, -2142015253, -483861416] [1529242730, -1451463575, 32679] [-2132277704, 1, -1, 147483648, 1926309171] [] \N [2085018568845570897, -9223372036854775808, -8896116716129220082, -1, -6134570] [-118956738, 32679, 245] [1, 1427715977, 0, 4, 1, 0, 0, 0] [-1930609070, 8, -1752919463] [] [0, -1086464093, 1, 0, 890911874, 1, -2, 1, 5, 1] [] [300.343000000000, -1.200000000000, 33.132400000000, 28.186500000000, 40.123000000000] [300.343000000000, 0.000000000000, 300.343000000000] [0.000000000000, 67.181500000000, 1.200000000000, 0.000000000000, 1.200000000000, -1.200000000000, 1.200000000000, 77.180000000000, 92.149500000000, 0.000000000000] [1.2000, 100.0200, 1.2000, 76.0046, -1.2000, 300.3430, 27.0752, -1.2000, 9.1459, 37.0804] [-1.2000, 0.0000, 0.0000, -1.2000, 300.3430, 300.3430, 19.0098, 0.0000, 32.0861, 0.0000] [100.0200, 1.2000, 28.0060, 8.0704, 46.0765] [96.0093, 40.1230, 300.3430, 40.1230, 88.0381] [1, 18, 6, 26, 29, 100, 1, 27] \N [40, 8, 300, 100, 40, 7, 100, 100, 40, -1] [300, 0, 100, 1, -1, 12, 100, 50] ["", "then", "弯弯的月亮顺驰没有万元漫长台式机"] ["y", "p", "没沈晓海快递新拓静静", "e", "我打印了想要地方", "询问企业管理一览表对不对短语日常", "小人", "hey"] ["well", "刚刚出去", "not"] ["?", "这部分任务磁带库追到回忆落实", "其中欧彩页就不万一会话天讯之间有限适用于", "彭伟", "小机器", "数据带", "p", "the", "him", "can't"] ["下图手里祺祥千金短句离谱", "?", "then did", "--", "接口"] ["hey", "准确", "", "所作", ""] ["you're her she see", "做事", "--"] ["j", "on because what it you of hey all good be", "--"] ["f", "l", "b", "v", "s", "r", "w", "c", "b", "v"] ["n", "f", "e", "v", "c", "v", "a", "t"] [] [] [2024-06-30, 2025-06-18, 2023-12-10, 2024-01-08, 2023-12-10, 2023-01-15, 2023-12-18, 2024-01-08] [2023-12-17, 2023-12-15, 2014-08-12, 2023-12-16, 2024-02-18, 2025-02-17, 2023-12-13, 9999-12-31, 2024-02-18, 2023-12-16] [2025-02-17, 2024-02-18, 2026-02-18] [2024-07-01, 2024-01-19, 2023-12-15, 2024-01-09, 2024-02-18, 2027-01-16, 2014-08-12, 2025-06-18, 2025-02-17, 2026-02-18] [2024-08-03 13:08:30, 2024-01-08 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2026-01-18 00:00:00, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00] [2024-01-31 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00] [2024-08-03 13:08:30, 2023-12-18 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00] [2023-12-11 00:00:00, 2023-12-14 00:00:00, 2024-01-17 00:00:00, 2023-12-16 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00] [2025-02-17 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00] [2023-12-09 00:00:00, 2024-01-09 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00] [2024-07-01 00:00:00, 2024-01-31 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00, 2023-12-13 00:00:00] [] [] [2024-01-31 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00] [2025-06-18 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00] [2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 2026-01-18 00:00:00, 2027-01-09 00:00:00] +231 1 0 -1610621063 147483648 2024-01-17 2023-12-16 2025-02-17 2017-03-18 -- 对你玉冰炒货选择周保全底价这几天 or can't was 加上又出宽屏牌子世上白明伟九洲中成份款 from your back in how so could 帮忙 ? 没结束预防订票彭巧语个月 1 1 300.343000 1.200000 300.343000 100.020000 2023-12-20 00:00:00 9999-12-31 00:00:00 2027-01-09 00:00:00 9999-12-31 00:00:00 2024-01-08 00:00:00 2023-12-17 00:00:00 2023-12-14 00:00:00 2024-01-09 00:00:00 [1, 0, 1, 1, 0] [0, 0, 0, 1, 1, 0, 0, 0, 1, 1] [1, 1, 1, 0, 1] [0, 1, 1, 1, 1, 1, 0, 1] [] [-43, 114, 127] [0, 1, -128, 18, 127, 0, 9, 1, -128, 3] [1, 15, 1] [-13645, -32768, -32768] [29247, 32767, 0, 749, -28005, -20764, 0, -21193, 28186, -1] [32767, 245, 245, -32768, 0] [-30848, 32767, -21111] [] [] [1, -181219644, 1261286038] [] [] \N [-1543503214132371158, -466791, -9223372036854775808] [] [0, -1, 674430040, -1, 1, 0, 0, -854384112] [0, -396129295, 0, 1, -772947999, -989670357, 1, -1] [] [1, 4, -1, 9, -1723990025, -1038972877, -1, 0] [70.069600000000, 14.125800000000, 1.200000000000, 42.030500000000, 100.020000000000, 40.123000000000, 91.081300000000, 0.000000000000] [40.123000000000, 40.123000000000, 10.114800000000, 300.343000000000, 40.123000000000] [59.025400000000, 300.343000000000, 0.000000000000, 1.200000000000, 64.048700000000, 100.020000000000, 100.020000000000, 99.028700000000] [] [] [33.1763, 40.1230, 27.1225, 100.0200, 69.0407] [100.0200, -1.2000, 40.1230] [1.2000, 40.1230, 0.0000, 78.1867, 16.1116] [1, 37, 40] [] [79, 45, -1, 100, 40, 40, -1, 100, 1, 40] [40, 100, 300, 40, -1, 51, 6, 300, 27, 300] ["陌生人鼎力青青绿色宏运合同会员卡", "v", "say him", "?", "didn't", "--", "这边", "-"] ["-", "?", "走过", "were could my her a ok", "?", "纱卡陈总就在呢里", "-", "the that not how you you then"] ["--", "h", "--", "赔钱英特一栏", "不给正运绿洲人已呵呵", "-", "?", "could", "have", "yeah were this they he's his get would"] [] ["", "?", "to but", "万能", "your a as he some when about was got why"] [] [] [] \N ["s", "n", "q", "u", "q", "u", "j", "j", "t", "b"] ["a", "v", "n", "w", "z"] [] [] [2024-01-09, 2023-12-14, 2023-12-12, 2023-12-16, 2024-02-18] [2025-06-18, 2023-12-09, 2023-12-18, 2024-02-18, 2024-01-08, 2024-07-01, 2023-12-10, 2024-01-31] [2023-12-14, 2024-06-30, 2025-06-18, 2023-12-11, 2026-01-18] \N [9999-12-31 23:59:59, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-01-19 00:00:00, 2024-08-03 13:08:30] [2026-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00] [2025-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-14 00:00:00] [2014-08-12 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2025-02-17 00:00:00] [2023-12-13 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00] [2023-12-12 00:00:00, 2027-01-16 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00] [2024-02-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00] [] [2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2024-01-17 00:00:00, 2023-12-11 00:00:00] [2024-07-01 00:00:00, 2023-12-10 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2026-02-18 00:00:00, 2024-01-09 00:00:00] [] +232 32679 1 -486021138 0 2024-02-18 2023-12-11 2024-06-30 2025-02-17 邮件没结束中华答案谁在回访 -- mean 令你容易深刻重点 got when you that's come her her go you going tell could so could go with -- \N 1 3.113100 1.200000 1.200000 85.105800 2023-12-16 00:00:00 2025-06-18 00:00:00 2025-02-18 00:00:00 9999-12-31 00:00:00 2024-02-18 00:00:00 2025-02-18 00:00:00 2026-01-18 00:00:00 2023-12-13 00:00:00 \N [] [1, 0, 1, 1, 0] [0, 1, 0, 0, 1, 0, 0, 1, 0, 1] \N [1, -74, 127, 21, -101, 9, 1, 32] [3, 1, -20, 0, 127, 1, 9, -1, -128, 0] [0, -123, 127] \N [-16999, -32768, -6182] [3490, 0, -9627, 1, 2032] [19514, 27435, -1, 16318, 20352] [147483648, -1646581711, 2147483647] [32679, 2012143619, -259078161, -1908335080, 2147483647, 1, -695346894, 32679] [-1840058572, 32679, -1181487441, 32679, -2147483648] [0, 1994865216, 32679] [0, 147483648, -8097908091795038685, -5456327, 469445572, 1415124073, 1, 0, -1779605, -95413229] [5686250733954094846, 9223372036854775807, 9223372036854775807] [-2062828273610548071, -1, 1826539310] [-1, 6834521, -7039494] [-1, -1162038910, 1, 0, -2, -2, 1, 0, 1, 1] [0, 3, 0, -2, 1, 1, 0, -1, -2, 0] [-1, 2, 0, -1872469302, 1, 0, 7, 1, -2, 0] [-1, 1, 0, 806171488, 1, -1, 0, -2, 1, 0] [] [90.100600000000, 40.123000000000, -1.200000000000] [0.000000000000, -1.200000000000, 95.070100000000, 300.343000000000, 40.123000000000, 100.020000000000, 300.343000000000, -1.200000000000, 300.343000000000, -1.200000000000] [40.123000000000, 22.120200000000, 95.054400000000, 19.169300000000, 40.123000000000, 59.044100000000, 83.044300000000, 79.148600000000] [40.1230, 49.1889, 40.1230, 53.1307, 40.1230, 40.1230, 38.1649, 300.3430] [0.0000, 23.1793, 300.3430, 7.1951, 1.2000, 300.3430, 61.0604, 100.0200] [40.1230, -1.2000, 0.0000, 0.0000, 0.0000, -1.2000, -1.2000, 300.3430, 1.2000, 11.0742] [27.1727, 100.0200, 18.0178] [100, 40, 300] [40, 55, 88] [] [] ["合兴创世纪交通借款零点马上有机会大河报", "--", "新版"] ["", "he will think time", "the", "at his me that's", "can have then that from been up the got at"] ["what", "now of she that's think how with from get", "-", "?", "贰仟健民结账刘雪掏点钱王全得网址"] [] ["", "just", "问起"] ["就有", "特此证明插拔红颜", "k"] [] ["基本上", "a", "they", "what as about do yes not for can't", "about", "你有智慧打电话", "then", "do mean right good me could your don't your is"] ["d", "t", "e"] \N [] ["f", "i", "g", "h", "i", "w", "b", "k", "k", "d"] [2023-12-20, 2027-01-09, 2023-12-20, 2025-06-18, 2007-09-25] [2025-02-18, 9999-12-31, 2024-01-19, 2024-02-18, 2025-06-18, 2024-02-18, 2023-12-10, 2024-02-18, 9999-12-31, 2026-01-18] [2026-01-18, 2024-01-31, 2024-06-30, 2023-12-12, 2026-01-18, 2027-01-09, 2027-01-16, 2024-07-01, 2027-01-16, 2025-02-17] [2023-12-11, 2027-01-16, 2023-12-16, 2014-08-12, 2023-12-15] [2024-08-03 13:08:30, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59, 2023-12-12 00:00:00, 2024-01-31 00:00:00] [2023-12-17 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59, 9999-12-31 00:00:00] [2025-06-18 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00, 2011-02-03 00:00:00] [9999-12-31 23:59:59, 2023-12-14 00:00:00, 2024-01-09 00:00:00] [] [9999-12-31 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2025-02-18 00:00:00, 2023-12-14 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00] [2014-08-12 00:00:00, 2023-12-20 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2003-04-11 00:00:00] [2024-01-19 00:00:00, 2023-01-15 08:32:59.123123, 2025-06-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-31 00:00:00, 2025-06-18 00:00:00] [2024-01-19 00:00:00, 2024-01-17 00:00:00, 2024-01-09 00:00:00, 2023-12-14 00:00:00, 9999-12-31 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2024-01-17 00:00:00] [2026-01-18 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00] [2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00] +233 -2147483648 -970886688 -456638967 -155625681 2023-12-10 2023-12-17 2023-12-14 2023-12-20 市委事业尽量图示占到四方控件 n been some if who would when i as I'll been of did when from -- 0 0 0.000000 1.200000 100.020000 300.343000 2023-01-15 08:32:59.123000 2023-12-10 00:00:00 2025-02-17 00:00:00 2024-08-03 13:08:30 2010-08-06 00:00:00 2023-12-18 00:00:00 2024-01-19 00:00:00 2023-12-11 00:00:00 [1, 1, 1, 0, 1] [] [1, 1, 1, 1, 1, 1, 1, 0, 0, 1] [1, 1, 0, 0, 1] [110, -103, 114, 13, 9, 49, 0, -1] [-1, 1, -70, 0, 9, -128, 1, -1] [] [] [32767, -21795, 1] [245, -7803, 32767, 1501, 31719, 32670, -32768, 30787, -32768, 245] [-14545, 245, 17050, 32767, -32768] [245, 0, -25952] [706017982, -1136348857, 32679] [147483648, -1, 891198593] [] [32679, -2147483648, -1, 337737749, -2147483648] [-4910831849351546193, 1, -9223372036854775808, 32679, -9223372036854775808, -2788319777592285771, -9223372036854775808, -1837779846556075837, 32679, -9223372036854775808] [2742918485483828364, -8028630003684184152, -9223372036854775808, -1, 245] [0, -9223372036854775808, -1125741388, 2019406628902256699, 9223372036854775807, -5265143548937838318, 9223372036854775807, 32679, 1891036977, 6224032228054231034] [0, 32679, 245] [1, 1, 9, 1, -1, -1270756578, -1, -2] [-874479034, 5, 0] [0, 0, -1, -1063191027, 6, 1, -2, 1, -2, 1] [0, 1, -1660107301] [1.200000000000, 1.200000000000, 0.000000000000, 0.000000000000, 1.200000000000, 97.184500000000, 18.090700000000, 32.042400000000] [0.000000000000, 1.200000000000, 100.020000000000] [20.196000000000, -1.200000000000, 68.170200000000, 58.195600000000, 100.020000000000, 40.123000000000, 0.000000000000, -1.200000000000] [1.200000000000, 74.163500000000, 0.000000000000] \N [16.1802, 100.0200, 0.0000, 5.0672, 80.0899] [] [50.1903, 16.1824, 21.0483] [95, 41, 40, 0, 1] [40, 0, 300, 1, 40] [100, 9, 34, -1, 40] [-1, 51, -1] ["get", "--", "?", "主演", "really"] ["there", "核算", "西郊", "-", "纯平陈老师报账确认徐州", "h", "一件", "", "r", "look why that's as"] ["me", "r", "?", "?", "her had as", "?", "安装计算跟单喝得多单单王青而是", "--"] ["谁家有货正常又是上午很深", "--", "武侠只要那行邮件比人磁盘翔龙", "k", "艺术", "?", "mean were go so how want that with", "here"] \N ["?", "自导分销商外聘埋在商人下雨算是刘畅所要求", "back"] [] ["重命名慧眼", "", "z", "it him did yes her then he were", "一线软件轻松在哪里呢元旦这中间相约", "to but", "其中磁带不让那行引进过来慈利达客运利万", "相逢", "then mean will ok are going she go that we", "?"] ["h", "v", "h", "y", "e"] \N ["f", "q", "j", "x", "z", "n", "j", "r", "s", "f"] ["z", "t", "l", "d", "b", "w", "a", "n"] [2024-01-31, 2023-12-15, 2024-02-18, 2024-08-03, 2014-08-12, 2023-12-17, 9999-12-31, 2024-01-08, 2023-12-17, 2024-01-09] [2023-12-10, 2025-06-18, 2024-08-03, 2023-12-11, 2024-02-18] [2023-12-15, 2002-11-23, 2023-12-09, 2024-02-18, 2024-06-30, 2024-02-18, 2024-08-03, 2014-08-12] [2024-01-17, 2024-06-30, 2024-02-18, 2026-02-18, 9999-12-31, 2024-01-17, 2025-06-18, 2023-12-13] [2023-12-16 00:00:00, 2014-08-12 00:00:00, 2027-01-09 00:00:00, 2026-01-18 00:00:00, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2023-12-10 00:00:00] [2025-02-18 00:00:00, 2023-01-15 08:32:59, 2023-12-19 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00] [9999-12-31 23:59:59, 2023-12-11 00:00:00, 2025-06-18 00:00:00] [2024-01-09 00:00:00, 2023-12-10 00:00:00, 2023-12-13 00:00:00, 2014-08-12 00:00:00, 2003-07-10 00:00:00, 2024-01-17 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00] [2024-01-19 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00] [2025-02-18 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00] [2023-12-19 00:00:00, 2024-01-08 00:00:00, 2023-12-15 00:00:00] [2024-01-19 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-12 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00] [2023-12-20 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2026-01-18 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 2016-06-15 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00] [2023-12-09 00:00:00, 2023-12-14 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-20 00:00:00] +234 -1 1 525553558 147483648 \N 2023-12-11 2024-01-19 2026-01-18 ? 不叫季度天网款且为你杂请你们蓝色服务器上出去押款 防治 get can't a my 问好需要北环这三大很好用 武汉整体取悦留一下三洋后期发吧选择 -- 符合上机柜做好有时候合同 1 0 300.343000 56.141500 0.000000 8.062100 2026-02-18 00:00:00 2026-02-18 00:00:00 2027-01-09 00:00:00 2023-12-18 00:00:00 2023-12-11 00:00:00 2014-08-12 00:00:00 2027-01-16 00:00:00 2023-12-20 00:00:00 \N [0, 0, 1, 0, 0, 1, 1, 1, 1, 1] [1, 1, 0, 1, 1, 1, 0, 0] [] \N [0, 1, -5, 0, 0, 4, -17, 127] [127, -61, -1] [] [18414, -16526, 16452] [32767, -32768, 0, 32767, -3015, -32768, -32768, 32767] [] [] [-1, 582173860, 2147483647, 2147483647, -1, 2041387067, 2147483647, 1187990124, 147483648, 809639061] [] [0, 32679, -1073204605, 0, -533208109, -1, 2147483647, 32679] [] [-9223372036854775808, 1, -1, 9223372036854775807, 245, 245, 32679, 245] [147483648, -2245684, -9223372036854775808, -1, 147483648] [1, 147483648, -9223372036854775808, 0, 147483648, -1, 9223372036854775807, 245, -9223372036854775808, -9223372036854775808] [32679, 1, -1, 9223372036854775807, -205813] [1, 5, 1, -2, 0, 390793694, -1, 0, -2, -987719127] \N [1, -133199114, 1] [-2, 1, 3, 9, 1] [1.200000000000, 100.020000000000, -1.200000000000, -1.200000000000, 17.172400000000, 100.020000000000, 300.343000000000, 1.200000000000] [72.002800000000, 24.161700000000, 30.037400000000, 67.161900000000, -1.200000000000, -1.200000000000, 40.123000000000, 300.343000000000, 83.134500000000, 1.200000000000] [100.020000000000, 10.018200000000, 1.200000000000] [300.343000000000, 100.020000000000, 0.000000000000, 61.032000000000, 300.343000000000, 16.096900000000, 100.020000000000, 300.343000000000] [300.3430, 47.0976, 40.1230, 100.0200, 40.1230] [-1.2000, 100.0200, 0.0000] [] [100.0200, 0.0000, 6.1805, 100.0200, 85.1503, 300.3430, 40.1230, 1.2000, -1.2000, 40.1230] [71, 300, 42, 3, -1, 73, 1, 55, -1, 26] [-1, 40, 95] [100, 70, -1] [54, 100, 8, 100, 72, 18, -1, 100, 40, 7] ["产品事业部彩霞天成苑总经理表示邦德方便", "when", "瑞恒艾蕊", "don't up yeah was oh no", "华林"] ["?", "say", "--", "can't", "back", "--", "?", "-", "k", "--"] [] ["if", "-", "all", "can't", "don't"] ["雪城踏踏实实饱和区号需问过", "or", "天宇检测参观以为你同志们圆圆明辉也得", "中心站", "w", "--", "y", "I'll you see a something get been because here", "--", "v"] ["?", "", "--", "世上这些话文字灵生欧典信天游合作过才能", "", "做生意丁老师一件上了清单郑州市人参在你是松伟", "", "-", "工作站", "代表"] ["?", "i", "代理价格便是报表你用手机吧同名美女底下制作", "because", "不对外", "say", "不对外第二款完美宝贵你有智慧打电话", "润泽"] ["with", "--", "小雨", "好好", "-", "he's", "做生意", "拿不到彩虹被授予"] ["j", "j", "y", "y", "q"] ["p", "w", "x", "p", "u"] ["w", "c", "y", "r", "i", "e", "f", "m"] ["v", "q", "x", "p", "a", "n", "u", "q", "t", "t"] [] [2023-12-14, 2027-01-16, 2023-12-13, 2023-12-10, 2025-06-18, 2024-02-18, 2024-01-19, 9999-12-31, 2023-12-14, 2024-06-30] [] [2025-02-17, 2014-08-12, 2023-12-09, 2025-06-18, 2024-01-31, 2027-01-16, 2023-12-16, 2023-12-12] [2023-12-15 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00] [] [2027-01-09 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00] [2023-12-14 00:00:00, 2023-12-20 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2026-01-18 00:00:00, 2011-10-12 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2025-02-18 00:00:00] [] [2025-06-18 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00] [2023-12-11 00:00:00, 2023-12-20 00:00:00, 2024-01-19 00:00:00] [2023-12-10 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00, 2008-12-24 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59.123000, 2024-08-03 13:08:30] [2023-12-10 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00] [2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00] [2026-01-18 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59, 2023-12-17 00:00:00, 2024-01-08 00:00:00, 2023-12-20 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00] [2023-12-10 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2024-01-09 00:00:00] +235 -1980731741 2147483647 201176375 -1877048503 \N 2024-02-18 2024-02-18 2025-02-17 关于微软支原体河南总代理对应删除包装 because how were in right tell with see really \N 星月发展拓展分数数哟看电影爸妈没钱 there you're out from time 0 1 100.020000 \N 300.343000 85.111500 2024-01-17 00:00:00 2023-12-11 00:00:00 9999-12-31 00:00:00 2024-01-08 00:00:00 2027-01-09 00:00:00 2023-12-20 00:00:00 2023-01-15 08:32:59.123123 2023-12-10 00:00:00 [1, 1, 1, 0, 0] [0, 0, 1, 1, 0, 0, 1, 1] [0, 1, 1, 1, 0, 0, 1, 1] [] [] [8, -1, 1, 72, 1, -61, 5, 118, 4, -1] [98, 1, 127] [] [32767, -1, -32661] [-27947, 32767, -2699, -8828, 25707, 2649, 9017, 32767] [19537, 13817, 11655, 1, -12707] [30589, -834, -2484, -1, -30863, 32767, 13893, 0, -32768, 1] [-819315433, -841217087, 563117122] [878165352, 1871515557, 2121858131, -2147483648, 1901572552, 0, -1, -715238371, 1960817989, 2081799658] [2147483647, 2147483647, -2147483648, -1629901413, -1, 1, 147483648, 1060957716] [-290849507, 2090020350, 111681633, -2105414029, 0, -1776106984, 32679, 2147483647, 1063256858, -2147483648] [147483648, 0, -3870912906946255941, 6098935746087286704, 4944749864732707812, 4318378764901116930, 9223372036854775807, 2231172287274754013] [1, 7100160182319429887, -6928347982151147906, 757445979200083699, 147483648, 0, 147483648, 1] [6125449867481097273, -8965706255963936392, 32679] [1905210265843529935, 147483648, 245] [5, 0, 1, -1462605162, 0, 2, -1, 1, 739763928, 0] [1, -1, -2, 1, 0, 1, 0, -2] [7, 1, 437079502, -1, 3, 3, 1930949864, 0, 0, -2] [-1998918255, 1, -1, 1, 1, -2, 0, 7] \N \N [43.162600000000, 300.343000000000, 7.058400000000, 47.020800000000, 78.177200000000, 300.343000000000, 17.103600000000, 1.200000000000] [-1.200000000000, 90.131800000000, 300.343000000000, 100.020000000000, 11.025800000000] [] [] [] [300.3430, 18.0943, -1.2000, 100.0200, 40.1230, 77.1890, -1.2000, 40.1230] [300, 40, 34, 94, 100, 1, 8, 62] [] [100, 300, 66] [6, 40, 300, 39, 40] ["can't", "有一天", "like", "", "e", "生意", "汉化令你为你春节", "-", "-", "yeah"] \N ["浩宇初次雅虎置业文利高压考验不问", "西郊出纳还有你的舞阳情况双机热备方案大公司经营赞美诗外运", "all who were in could okay as is had didn't", "?", "all no could had she we", "接收", "中环海川冰火文艺四川不分下个礼拜黄黎明寻找有句话", "f", "高端", "都用和兴提呢王枫咩女英莲美丽农行"] ["就算我那个叫回你外教轩明向西加为好友切换几天就走迅盘", "", "?", "what my", "v", "from that", "it", "w", "--", "he they are no in"] ["", "?", "广益", "--", "--", "w", "we just see you're", "预算"] ["?", "got going then a want there he's about that now", "back", "", "he"] ["s", "山特没沈晓海不愿意", "", "邮箱财务太不主机关闭一脚新普前台管理", "been out one and you we in he she", "签名提醒强暴", "--", "m"] ["to there are think hey", "got is time", "?", "-", "to it's going yes oh with say"] ["v", "a", "u", "q", "j"] ["g", "a", "a", "h", "z", "v", "l", "s"] [] ["k", "s", "a", "p", "q"] [2023-12-19, 2024-02-18, 2024-01-09, 2027-01-16, 2024-02-18, 2024-01-08, 2023-12-10, 2023-12-17] [2025-02-18, 2023-12-15, 2023-12-11] [2024-01-08, 2024-01-09, 2024-01-08, 9999-12-31, 2024-02-18] [2027-01-09, 2023-12-17, 2014-08-12, 2023-12-20, 2025-02-17, 2025-02-17, 2027-01-09, 2023-12-19, 2023-12-11, 2025-02-17] [2026-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00] [2023-12-11 00:00:00, 2018-01-06 00:00:00, 2024-01-08 00:00:00] [2025-06-18 00:00:00, 2023-12-13 00:00:00, 2023-12-20 00:00:00] [9999-12-31 23:59:59, 2023-12-15 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00] [2023-12-14 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-13 00:00:00] [2024-01-09 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00, 2023-12-11 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2024-01-08 00:00:00] [2014-08-12 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00] [2024-01-17 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00] [2023-12-11 00:00:00, 2025-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00, 2018-06-19 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00] [2025-06-18 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00] [2023-12-15 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00] [2024-01-19 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00, 2023-12-19 00:00:00, 2025-02-18 00:00:00] +238 432963337 32679 678420412 147483648 2026-02-18 9999-12-31 2023-12-12 9999-12-31 接收名次面包一新需方离开反正手下 don't so k that's ok the him in just so oh 稍后 控制卡多久海涛辅佐马后电厂名杂盈盈明白 i my yes about been oh can't 1 1 40.123000 1.200000 79.182700 300.343000 9999-12-31 23:59:59 2023-12-12 00:00:00 2024-07-01 00:00:00 2027-01-09 00:00:00 \N 2025-06-18 00:00:00 2023-12-15 00:00:00 2027-01-09 00:00:00 \N [1, 1, 0, 0, 1, 0, 1, 0, 1, 0] [] [1, 0, 0, 0, 1, 0, 0, 0, 0, 0] [0, 8, -6] [53, -43, 127, 103, 31] [0, -128, 111, 1, 127] [1, 7, 127] \N [] [32767, 245, 576] [-32768, 26011, 244, 1, -32093, 12017, 1, 32767] [-872400198, 1079271260, 147483648, 32679, 1432915763] [1580882287, -57364086, -2147483648, 147483648, 1079529792, 697097364, -2147483648, 2147483647, -251536364, -1230369975] [-2147483648, 0, 483465563] [] [] [147483648, 147483648, -1, -9201117066545772144, 245] [9223372036854775807, 32679, 32679] [9223372036854775807, 32679, -2009369864, 1332089184, -1, -9223372036854775808, -9223372036854775808, 7104190951405486397, -1, 245] [-2, 1, -2] [2, -2, -2, -1, 1911746092, 0, 0, 1] [] [-771507573, -1, 3] [20.048900000000, 1.200000000000, 60.011200000000, 72.183000000000, 100.020000000000, 100.020000000000, 1.200000000000, 100.020000000000] [1.200000000000, 1.200000000000, -1.200000000000, 0.000000000000, 32.192200000000, 11.102100000000, 0.000000000000, -1.200000000000] [40.123000000000, 40.123000000000, 40.123000000000, 100.020000000000, 40.123000000000] [1.200000000000, 1.200000000000, 17.107700000000, 40.123000000000, 93.144100000000] \N [0.0000, 40.1230, -1.2000, 93.1102, 0.0000, 65.1251, -1.2000, 40.1230] [40.1230, 94.0925, 88.1661, 100.0200, 0.0000, 1.2000, 300.3430, 40.1230] [] [89, 1, -1, 22, 0, 1, 57, -1] [100, -1, 0, 35, 300] [] [-1, 24, 23, 1, 69, -1, 100, -1, 0, 56] [] ["been back from can", "--", "b", "--", "单太惭愧那行管理创新看完质保代码汇集木马"] ["know okay I'm", "侯玉那行老是时候前面一米", "-", "我就", "说好百脑汇结账江河有待于有没有透明十二婚嫁", "痛苦", "you", "晚上聊"] ["he's", "代理", "-"] ["各位设备", "can", "me been your this his say"] ["that like did be for", "y", "as don't with they come this ok", "don't all then will", "?", "-", "yes", "t"] ["郑东", "", "三阳开泰那样子晚上打你给上演", "--", "?", "卫华中庸青云水晶无论老了天天散落的显示", "y", "ok", "about", "some see okay was had don't at on you this"] ["here me if he's", "", "不可", "?", "--"] ["d", "c", "r"] ["q", "l", "u", "l", "n", "d", "w", "o"] ["t", "g", "f", "x", "i", "w", "q", "o", "j", "x"] ["o", "m", "b", "z", "l", "n", "n", "d", "p", "m"] [2023-12-14, 2023-01-15, 2023-12-10, 2024-01-31, 2023-12-18, 2024-08-03, 2026-01-18, 2023-12-19] [2023-12-09, 2023-12-14, 2023-12-17, 2024-08-03, 2025-02-17, 2024-06-30, 2024-01-31, 2023-12-13] [2025-02-18, 2023-12-09, 2025-02-18, 2024-02-18, 2027-01-09, 2025-06-18, 9999-12-31, 2024-08-03, 2014-08-12, 2023-12-12] [2024-02-18, 2027-01-16, 2023-12-17, 2019-05-21, 2024-02-18, 2027-01-09, 2024-02-18, 2024-01-09] \N [] [2024-02-18 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 2024-06-30 12:01:02] [] [2024-01-31 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00] [2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-18 00:00:00] [2023-01-15 08:32:59.123000, 2023-01-15 08:32:59.123000, 2025-02-17 00:00:00, 2005-09-13 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59.123000] [] [2024-01-19 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-13 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2008-05-23 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-16 00:00:00, 2023-12-14 00:00:00, 2027-01-09 00:00:00] [2023-12-18 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2026-01-18 00:00:00] [] +239 -970156960 1313374001 147483648 -2147483648 2024-01-08 9999-12-31 2023-12-18 9999-12-31 o 好好哦啊答案吃住国际企业交到也有 with good got - 空间 - 1 0 47.091000 30.120400 0.000000 52.179600 2023-12-10 00:00:00 \N 2024-07-01 00:00:00 2024-01-09 00:00:00 9999-12-31 00:00:00 2024-01-08 00:00:00 9999-12-31 23:59:59 2024-01-19 00:00:00 [] [0, 0, 0, 0, 0, 1, 0, 0] [1, 1, 1, 0, 1, 0, 0, 1] [1, 1, 1] [1, 54, -85, 1, 3] [0, -42, -1] [24, 0, 5, 1, -119] [-19, 4, -122, -84, 0, -128, 7, -1] [22509, -5013, 21418, -31609, -32768, -23749, 0, 245, -32593, -20168] [30946, -32768, -1, 245, 30374] [29237, -1, -7785, 245, 1, -9738, 13667, -32768] [] [32679, 1, 241291277] [-1618380144, 842418170, -626130000] [0, 32679, -2147483648] [1186958825, 2147483647, -1, 1096192341, 1774452784, 32679, 1, -886288028] [-1010151468485058032, 32679, 1, -9223372036854775808, 4754238, -5373789906582560572, -1118105157, -1, 245, 0] [-8923696460727149085, 147483648, 9223372036854775807, 1904241361, 245] [32679, 31340038834115158, 0] [0, -1, -2047300157592192826] \N [8, -2, -1] [] [1, -2, 1, 0, 1] [40.123000000000, 1.200000000000, 100.020000000000, 12.142300000000, 0.000000000000, 81.168700000000, 53.135700000000, 92.099300000000, 14.003800000000, 300.343000000000] [24.092100000000, 1.200000000000, -1.200000000000, -1.200000000000, 23.163300000000] [1.200000000000, 300.343000000000, 34.074700000000, 0.000000000000, 18.051300000000, 300.343000000000, 0.000000000000, 65.092400000000, 0.000000000000, 1.200000000000] [100.020000000000, 40.123000000000, 52.003500000000, -1.200000000000, 96.079700000000, 300.343000000000, 100.020000000000, 40.123000000000, 100.020000000000, -1.200000000000] [0.0000, 100.0200, -1.2000] [40.1230, 40.1230, 300.3430, 40.1230, 40.1230, 5.0445, 0.0000, -1.2000] [34.1563, 64.0218, 1.2000, 40.1230, 38.1012] [-1.2000, 66.1913, 95.1997, 1.2000, 0.0000] [38, 100, 49, 0, 1, 40, 1, -1] [300, -1, 40] [] [-1, 52, 100, 20, 100, 300, 27, 100] ["h", "标准天成苑打扰业茂含义相见点事", "he", "--", "心里"] [] ["咨询", "格瑞", "--", "like I'm I'm or go to who", "--", "of been they", "I'll", "hey her he's good"] ["?", "昨晚上哪一步懂得科美库房资讯有关大学路世安地方那个", "追求", "all from", "from"] ["been", "-", "g"] ["-", "?", "?", "?", ""] ["到来", "--", "he is I'll", "now", "go"] [] [] ["u", "i", "z", "y", "t"] ["f", "n", "y", "g", "q", "z", "n", "f", "l", "k"] ["h", "c", "t", "d", "b", "p", "p", "d"] [2023-12-15, 2014-08-12, 2026-01-18, 2024-07-01, 2024-02-18, 2023-12-11, 2026-02-18, 2024-01-09, 2023-12-10, 2023-12-19] [2023-12-16, 2024-02-18, 2027-01-16, 2023-12-14, 2023-12-17] [2023-12-12, 2024-06-30, 2023-12-16, 2023-12-17, 2023-12-14] [2023-12-19, 2024-01-09, 9999-12-31] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [] [2024-01-09 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2011-12-08 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-15 00:00:00] [2023-01-15 08:32:59, 2002-05-03 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-18 00:00:00, 9999-12-31 23:59:59] \N [2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00] [2024-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00] [2027-01-16 00:00:00, 2023-12-15 00:00:00, 2023-12-17 00:00:00, 2024-01-19 00:00:00, 2023-12-10 00:00:00] [2026-01-18 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00] [2024-01-08 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-08-03 13:08:30, 2025-02-18 00:00:00, 2023-12-16 00:00:00] [2023-12-17 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00] +240 \N 0 1927599908 604105100 2018-11-26 2023-12-17 2025-02-17 2024-01-09 now her 经历过 \N 合作 王东亮 look q ? \N 1 40.123000 27.044600 40.123000 37.190400 2023-12-16 00:00:00 2023-12-17 00:00:00 2023-12-11 00:00:00 2023-12-10 00:00:00 2023-12-14 00:00:00 2023-12-09 00:00:00 2024-01-31 00:00:00 2023-12-13 00:00:00 [0, 1, 0, 0, 1, 0, 0, 0] [0, 1, 1, 1, 1] [0, 1, 0, 1, 0] [0, 0, 1, 1, 0, 1, 1, 1, 0, 0] [38, -128, 1, -1, 127, 0, 61, 41] [-128, 100, -1] [-1, 0, 51, 47, 0, -111, -57, -88] [-26, 0, 0, -128, 1] [-21181, 23333, -32768, -22151, -31311, 32767, -848, -32768, 13985, -25702] [-1, 5126, -1] [-23177, 0, 32767, -7517, 0] [-25923, -24222, 245, 245, 16240, 245, 1, 31409] [2147483647, 2147483647, -1264010389, -1, 1, 781797221, -1032243046, -295461898, -2147483648, 147483648] [505571145, -1589761716, 817621538, 2147483647, 147483648] [1, 147483648, 32679, 1837342985, 2147483647] [] [-5908074254617199952, -1, 2221849318485505421, 245, 0, -1, 9223372036854775807, 32679] [-751990619, 49145294396147615, 9223372036854775807, 245, 1, 0, 147483648, 32679, 0, 9223372036854775807] [-1, 4021104764190928288, -8576038841844301480, 245, -452523879, -1, -2647537897682990122, 147483648] [1, 32679, -1, 147483648, 32679, 245, -9223372036854775808, -4254702771797751916] [1, 0, -2, -776419513, 4] [-2, 1, 0, -670598076, -2, 1, 4, -2, 0, 0] [] [] \N [1.200000000000, 300.343000000000, -1.200000000000, 100.020000000000, -1.200000000000, 33.095800000000, 40.123000000000, 40.186600000000] [90.140000000000, 30.175400000000, 0.000000000000, 30.111400000000, 300.343000000000, 19.059500000000, 100.020000000000, 100.020000000000] [] [10.1812, 1.2000, 100.0200, 1.2000, 0.0000, 54.0611, 81.0785, 25.0148] \N [5.0176, 300.3430, 40.1230] [100.0200, 300.3430, 70.1049] [300, 53, 300, 100, 0, 300, 43, 68, -1, 58] [1, 42, 5, 100, 31, 1, 1, 56, 300, 1] [300, 100, 300, 40, 26] [1, 35, 100, 40, 300] \N \N ["华康", "-", "had", "just on no right a so don't going are", "--", "-", "want", "think you're you're", "just", "--"] ["she something", "they okay I'll in for", "?", "-", "-", "from I'm", "w", "泥石谁南村个人小精灵这个"] ["want this know come your was as her", "get", "-", "she really been who", "see", "ok got", "--", "?", "", "--"] ["-", "文艺他在哪偶然追求嵩山大领导岩春保证有缘", "-", "something", "-", "they would with", "", "", "?", "one so been can or know that's could yeah really"] ["t", "right be mean you up can then could", "if"] [] [] ["e", "l", "w"] ["p", "l", "t", "i", "p"] ["s", "i", "x", "q", "h", "k", "y", "i", "n", "r"] [2023-12-17, 2025-02-17, 2025-06-18, 2024-01-31, 2023-01-15] [2023-12-13, 2023-12-18, 2023-12-16, 2024-01-09, 2023-12-12, 2024-01-08, 2023-12-10, 2023-12-12] [2023-12-09, 2025-06-18, 2027-01-09, 2024-02-18, 2023-01-15, 2023-12-17, 2024-08-03, 2025-02-18] [] [2024-08-03 13:08:30, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-15 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00] [2026-02-18 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00] [2023-12-16 00:00:00, 2023-12-17 00:00:00, 2024-01-17 00:00:00, 2024-01-09 00:00:00, 2024-01-08 00:00:00] [2024-01-19 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00] [9999-12-31 23:59:59, 2016-02-18 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] [2023-12-16 00:00:00, 2023-12-11 00:00:00, 2023-12-11 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30] [2024-08-03 13:08:30, 2024-01-19 00:00:00, 2024-01-08 00:00:00, 2019-12-09 00:00:00, 2027-01-16 00:00:00, 2027-01-16 00:00:00, 2023-12-18 00:00:00, 2023-12-14 00:00:00] [2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2023-12-09 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123000] [2023-12-09 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00] [2025-06-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2013-10-11 00:00:00, 2023-12-15 00:00:00] [2024-08-03 13:08:30, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00] [9999-12-31 23:59:59, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00] +241 -860414128 32679 -1089459909 2147483647 2024-08-03 2023-12-14 2026-02-18 2023-12-16 I'm the back your his had -- -- 开封退信早点网讯没有办法不要图像熟悉赵洪应才能 m -- like I'm and here to \N 0 3.140100 0.000000 33.190900 15.102000 2023-12-13 00:00:00 \N 2023-12-15 00:00:00 2023-01-15 08:32:59.123000 2023-12-17 00:00:00 2027-01-16 00:00:00 2024-01-19 00:00:00 2023-12-11 00:00:00 [] \N [] [] [37, 0, 106, 0, 8] \N [0, 87, -112, 0, 13, 24, -1, -128, -60, -1] [-64, -110, 49, 127, 0, 8, 1, 127, -1, 1] [14604, 0, -16809, -32768, -20693, 20276, -10807, 32767] \N [-15115, 29356, 32767, -32768, -14188] [16066, 28275, 32767, -32768, 19863, 32767, -32768, -15906] [-2147483648, 1, -792012973, -387042906, 1346150117, 0, -2147483648, -2147483648, 874212762, 32679] [-1358847999, 307738508, -1, -1, -2147483648, -2006070426, 1680672010, 147483648] [0, -295898789, 1412569398, 32679, 1, 147483648, -1933870029, 0, -572298737, 2147483647] [32679, 1544210303, -1499177487] [1310059103, 9223372036854775807, 3305263, 9223372036854775807, 32679] [-9223372036854775808, 0, 9223372036854775807] [-1, 245, 32679, -3924568779148058787, 9223372036854775807] [-1, 32679, 245, 482066824, -2974845802131683144] [7, 2, 1, -1, 9] [0, 1050916650, -2] [-1, 1, 1, 2099174450, 0] [-2, 0, -2, -2, 0, -144141697, -2, 1483695253, -1, 1] [300.343000000000, 100.020000000000, 1.200000000000, 100.020000000000, 300.343000000000] \N [40.123000000000, 40.123000000000, 100.020000000000, 40.123000000000, 40.123000000000, 300.343000000000, 30.120900000000, 300.343000000000, 300.343000000000, 300.343000000000] [] [100.0200, 1.2000, 40.1230] [100.0200, 0.0000, -1.2000, 1.2000, 100.0200, 100.0200, 0.0000, 300.3430] [62.0146, 71.1026, 1.2000, 96.1277, 83.1231, -1.2000, -1.2000, 1.2000] [47.0009, 56.0908, 100.0200] [0, 19, 87, 40, 40] [13, 95, 86, 27, 100, 0, -1, 1, -1, 40] [] [] [] \N [] ["time", "c", "we I'll what mean think well didn't", "is", "of", "go now going no your okay I'm your good", "计算睡觉价格传美安小验货单", "I'll", "参与", "妈的资格查收方面说不清楚椅子上企业管理一贯淡季"] ["--", "备注", "his about up as this one"] ["?", "?", "?"] ["?", "this", "?", "who", "代理透露认可这话科美制动阀", "合同", "挺多同辉稳定效果广场银联礼拜一最重要", "小型机", "工程简单", "百脑汇这一生封库"] ["out then in he's at", "up", "back can't he's got and don't up him", "--", "h"] ["r", "y", "q", "b", "x", "o", "v", "d", "i", "j"] ["s", "u", "z"] [] ["v", "b", "e", "t", "x"] [2024-02-18, 2024-07-01, 9999-12-31, 2006-08-03, 2023-12-18, 2023-12-11, 2024-07-01, 2025-02-18] [] [2024-02-18, 2024-08-03, 2024-02-18, 2023-12-17, 9999-12-31] [] [9999-12-31 00:00:00, 2014-08-12 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02] [2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00, 9999-12-31 00:00:00, 2003-07-04 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [] [2023-12-15 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30] [2017-08-05 00:00:00, 2023-12-19 00:00:00, 2023-12-11 00:00:00, 2023-12-18 00:00:00, 9999-12-31 00:00:00] [2023-12-14 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2007-02-07 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00] [2025-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2023-12-10 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00] [2023-12-18 00:00:00, 2010-01-08 00:00:00, 2023-12-11 00:00:00] [2024-01-17 00:00:00, 9999-12-31 00:00:00, 2023-12-13 00:00:00] [2026-01-18 00:00:00, 2023-12-17 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2024-07-01 00:00:00] [2023-12-14 00:00:00, 2024-01-09 00:00:00, 2023-12-20 00:00:00, 2024-01-31 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] +242 414718924 -53984047 0 -2147483648 2024-02-18 2023-12-15 2026-02-18 2023-12-11 赶紧王之长期工作站扩展名都还操作系统向此 \N \N 帮忙相处钻石心里金冠 多些 - -- 1 1 100.020000 300.343000 100.020000 1.200000 2027-01-09 00:00:00 2024-08-03 13:08:30 2025-02-18 00:00:00 9999-12-31 00:00:00 2024-01-08 00:00:00 2026-02-18 00:00:00 2023-12-16 00:00:00 2023-12-18 00:00:00 [0, 1, 0, 1, 1, 1, 0, 0] [0, 0, 0] [0, 1, 0, 0, 1] [0, 0, 0, 0, 1, 0, 0, 1] [0, 1, -1, 1, -1, -128, 4, 127] [-128, 0, -128] [1, 17, -4, 3, -5, 118, 0, 113, 127, 37] [0, -128, -128, -1, 0] [-20758, -3673, -8862] [25617, 245, 1, 245, 32767, 245, 1, -21404, -2438, 0] [] [] [1, 1690807945, -1671522457] [2147483647, 32679, -1527670807] [1255504371, -1672898372, 1136344919] [624473152, -2147483648, 1, 958746346, 1] [] [245, -9223372036854775808, 9223372036854775807, 1, 32679, -6059232, 1, 1, 32679, 147483648] [-1, 6380343450576440052, -9223372036854775808, -6925142134505387452, -179331368632195634, -9223372036854775808, -9223372036854775808, 3716496275684364670, 1, -5545740539712586501] [32679, -1, -1] [-1, 0, 0, -1, 1, 1368628465, 1, 8] \N [] [-1606128338, -2, -1] [300.343000000000, 85.018800000000, 40.123000000000, 40.123000000000, 19.051800000000] [96.161500000000, -1.200000000000, 100.020000000000] [0.000000000000, 100.020000000000, 40.123000000000, -1.200000000000, 40.123000000000] [] \N [0.0000, 1.2000, 0.0000, 0.0000, -1.2000, 0.0000, 300.3430, 1.2000] [78.0018, 100.0200, 40.1540, 300.3430, 100.0200] [93.0459, -1.2000, 49.1245, -1.2000, 33.0735] [1, 0, 40, -1, 1] \N [47, 40, -1, 40, 300, 40, 80, 1] [40, 4, 34] ["那一片惊风管理文杰黑色王老师屏幕东大街从事正常", "got", "is out"] ["--", "--", "词库均为英联五星全系列均为取回金蝶", "--", "it oh can't it's"] ["--", "协奏曲", "-", "was", "翔龙以后心思大厦排列软件园安小懂吗白菜", "-", "?", "比人"] ["", "", "okay", "?", ""] ["q", "?", "?", "him", "", "毕业墙隔壁冤枉承认大话没沈晓海一下阿拼搏", "then", "就不", "?", "g"] ["me", "a", "?"] ["玲玲回答明星广告第六不好累了城市", "-", "--", "", "h"] ["been time could you think there with", "no", "我不", "?", "此外"] \N ["x", "u", "t", "f", "t"] ["u", "s", "j", "d", "l", "e", "x", "s"] [] [] [2025-02-17, 2023-12-18, 2024-02-18, 2024-01-08, 2024-08-03, 2025-02-17, 2023-12-16, 2014-08-12, 2024-02-18, 2024-02-18] [2023-12-14, 9999-12-31, 2025-06-18, 2023-12-18, 2027-01-09, 2023-12-14, 2024-06-30, 2023-12-19, 2023-12-16, 2017-10-27] [] [2014-08-12 00:00:00, 2024-07-01 00:00:00, 2023-12-16 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00] [2025-02-17 00:00:00, 2023-12-16 00:00:00, 2024-01-19 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-01-15 08:32:59] [2023-12-13 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00] \N [2023-12-18 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00] [2023-12-17 00:00:00, 2025-02-18 00:00:00, 2024-01-19 00:00:00] [9999-12-31 23:59:59, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2024-01-09 00:00:00, 2023-12-17 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2024-01-19 00:00:00] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2027-01-09 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2023-12-17 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2023-12-18 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2026-01-18 00:00:00] [2023-12-20 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00] [2024-01-19 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2023-12-20 00:00:00, 2006-04-04 00:00:00] +246 \N \N -2147483648 -1 2023-12-10 9999-12-31 9999-12-31 2023-12-19 be see - z -- - him 1 1 57.139900 40.123000 -1.200000 0.000000 2026-02-18 00:00:00 2024-01-17 00:00:00 2023-12-11 00:00:00 2024-06-30 12:01:02.123000 2023-12-09 00:00:00 2025-06-18 00:00:00 2024-01-09 00:00:00 2023-12-17 00:00:00 [1, 1, 1, 0, 1, 0, 0, 0, 0, 1] \N [1, 0, 0, 0, 0, 0, 0, 1, 0, 1] [1, 1, 0] [58, -1, -1, 127, 1, 1, -1, -128, -63, 8] [-128, 0, -128, 127, -112, -128, -128, -128, -1, 27] [5, -128, 9, -128, -128] [] \N [] [245, 32767, 26941] [] \N [2147483647, 1195033464, 147483648, 1, 0] [1010303455, 2147483647, -583306503, -21654348, -1130011188, -1, 32679, -2147483648] [2147483647, -830855963, 0, 32679, 597419139, 2147483647, 499851362, 820691196, 1183230369, -1] [9223372036854775807, -6433950123756859864, -9223372036854775808, -6047471507947963323, -1] [32679, 1, -1, 6369059, 147483648, 32679, 3140512, -5971852209257458274] [8880702468871500304, 147483648, 575317523, 32679, -1118050590, -1818682549, 3152534199436769115, -6467991230791599294] [147483648, 9223372036854775807, -7678999356184627524, 147483648, 147483648, 32679, -1, 7867073, 1724310, -9223372036854775808] [1, -2, 7] [-2, -1, 1, 2, 0] [1, -2, -2, 6, 0, 8, -2, -2] [] [92.120700000000, 0.000000000000, 300.343000000000, 0.000000000000, 300.343000000000, 0.000000000000, 25.134800000000, 63.131000000000] [100.020000000000, 1.200000000000, 40.123000000000, 15.008300000000, 0.000000000000] [100.020000000000, 0.000000000000, -1.200000000000, 95.013100000000, 69.170400000000] [44.169100000000, 81.035400000000, 100.020000000000] [100.0200, 9.1743, 40.1722, 1.2000, 300.3430, 300.3430, 40.1230, 300.3430] [] [-1.2000, 22.1835, -1.2000] [] \N [35, 85, 60, -1, -1] [-1, 59, 0] [67, 61, 0, 85, 300] ["-", "for could just really don't now", "he's some here just or are like could him", "--", "can", "e", "-", "not or going had the right because were about"] ["here", "", "that's on a got okay now he hey and", "-", "分析", "one", "规模万方咨询服务对面", "one that's"] [] ["can't", "e", "what it's just some get but be oh it's"] \N ["我打印", "were", "--", "?", "f"] ["沟通硕博热情劲", "m", "i", "would", "激情", "创恒电子", "have are", "well out who they be will see but from", "-", "签单收藏夹忽悠晚上聊做完承诺质量领取苦楚"] ["so", "", "q", "改写党经理乔总不敢奋斗小型机摩擦拉美刘广", "h", "got that's who that's would well what", "right", "oh all out at don't well", "just", "been they my about will"] ["p", "z", "k"] ["o", "j", "n", "r", "r", "q", "c", "m"] [] ["w", "d", "y", "i", "x"] [2024-01-09, 2023-12-10, 2024-02-18] [2025-02-17, 2023-12-16, 2006-05-23, 2023-12-12, 2024-01-19] [2023-12-15, 2023-12-20, 2023-12-16, 2025-06-18, 2026-01-18] [2025-06-18, 2024-01-31, 2025-06-18, 2024-01-31, 2025-02-17, 2024-02-18, 2024-01-09, 2025-02-17] [2024-01-08 00:00:00, 2023-12-09 00:00:00, 2014-08-12 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2027-01-16 00:00:00, 2025-02-18 00:00:00, 2027-01-09 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00, 2025-02-17 00:00:00] [2027-01-09 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [2014-08-12 00:00:00, 2024-06-30 12:01:02, 2023-01-15 08:32:59, 2024-08-03 13:08:30, 2000-10-01 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00] [] \N [2023-12-16 00:00:00, 2023-12-19 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 9999-12-31 00:00:00] [] [2023-12-12 00:00:00, 2023-01-15 08:32:59.123123, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00] [2024-01-19 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00, 2023-12-14 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30] [] [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30] +247 32679 147483648 0 -1 2024-02-18 2023-12-20 9999-12-31 9999-12-31 开会诚心单条欧阳睡觉他妈区号院内 would some can't a can his up something tell not would come would were 孔艳本科实现航务饲料看你补上情况个哦 删除 联恒 ? ? 0 0 \N 13.025000 21.074200 1.200000 2023-12-19 00:00:00 2025-02-18 00:00:00 2023-12-20 00:00:00 2025-06-18 00:00:00 2024-01-31 00:00:00 2024-01-19 00:00:00 2024-01-17 00:00:00 2025-06-18 00:00:00 [1, 0, 1, 0, 0, 1, 1, 0, 0, 0] [0, 0, 0, 0, 0] [0, 0, 1, 0, 1] [1, 0, 0, 0, 0, 0, 0, 0, 1, 1] [-1, 3, -1] [1, 31, -81] [69, 122, 0, 0, -97, 1, 3, 9] [0, 1, -128, 1, 82, -1, -128, -113, 127, 1] [] [] [-16871, 11678, -14978, -15896, -18761, 245, 14023, 0, 30341, -1] [1, -1039, 22563, 2178, -1, -31599, -1, -21963, -32768, -12035] [-1, 1154277069, 1321867227, -2147483648, -517698943] [] [-913690075, -2147483648, 1, 0, -1425006900] [1837358028, -1, -401728375] [-1562878, 0, 3925661, -6805464179833929842, 5456920212010660566, -8170598, -7992006, 9223372036854775807, -4118341197457978954, 625496875318247921] [-681268214, 147483648, 9214800145884178768, 147483648, -9223372036854775808] [-805825599594807633, 32679, -3752860464478402893, -3633108251017324001, 32679] [] \N [1, 0, 1] [] [-1, -1845046425, 0] [-1.200000000000, 85.000100000000, 3.078800000000, 300.343000000000, -1.200000000000] [0.000000000000, 7.077500000000, 0.000000000000, -1.200000000000, 0.000000000000, 60.146000000000, 100.020000000000, 57.167700000000] [-1.200000000000, 300.343000000000, 1.200000000000] [47.129500000000, 73.028200000000, 0.000000000000, 28.167900000000, 40.123000000000] \N [0.0000, 0.0000, 40.1230] [] [300.3430, 1.2000, 0.0000] [33, 16, 45, 300, 300, 79, 300, 1, 1, 0] [40, 25, 100] [] [] ["good not can't", "之夜电池记录分开没沈晓海性质能力显卡充满", "这场", "my me here can't say I'm we of", "?", "--", "--", "招工"] ["集成", "the", "半年", "--", "处理年前天地", "hey be here what can't come a oh", "奇偶部队", "-", "", "影片个哦忘记不了燕龙订票女朋友财务部领导也要一级"] [] [] ["it", "are", "", "yeah", "one", "谈判", "e", "好了谁那里动手脚看到片发过图片中原路次要", "王燕纯度", ""] ["利落", "打我同时", "?", "from me could my", "需问过", "最好摆放地区过的好吗争取回复风泽园", "the mean been because something", "自动", "up of know at do because was going think", "科技大厦小妹初五方案"] [] ["一段", "s", "w", "well when time oh", "his mean we", "?", "good", "看是一条线雷林说句一群上午"] ["f", "o", "j", "c", "g", "o", "b", "j", "g", "v"] ["e", "p", "a", "s", "a"] ["o", "z", "j"] ["k", "x", "q"] [9999-12-31, 2026-01-18, 2024-01-19, 2027-01-16, 2026-02-18, 2023-12-09, 2014-08-12, 2023-12-12] [2027-01-09, 2024-07-01, 2026-01-18] [] [] \N [2024-02-18 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00] [2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00] [] [2026-02-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [] [] [] \N [2027-01-16 00:00:00, 2023-12-14 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-11 00:00:00, 2025-02-17 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00] [2023-12-14 00:00:00, 2023-12-16 00:00:00, 2024-01-17 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00] [] +248 2147483647 1457374765 1603180634 -2147483648 2023-12-13 2027-01-09 2024-01-17 2024-08-03 o my 一辈子 \N is I'm can't can't all when be at if something as 顺河路 信用社 \N 1 16.052800 -1.200000 0.000000 0.000000 2023-12-15 00:00:00 2027-01-09 00:00:00 2023-12-14 00:00:00 2023-12-16 00:00:00 2023-12-16 00:00:00 2023-12-20 00:00:00 2024-01-17 00:00:00 2023-12-20 00:00:00 [1, 1, 0, 1, 0, 1, 1, 0, 0, 0] [1, 0, 0, 0, 1] [1, 0, 1, 1, 1, 0, 1, 1, 0, 0] [1, 1, 0] [] [] [] [-3, -16, -1] [9285, 245, 245, 32582, -1] [-2411, -1, 21133, 29164, 1, -1, 245, 32767, 1, -1] [] [32767, 245, 25596, 4176, 980] \N [-2147483648, -2147483648, -1830835671, 2147483647, 540740001] [877284352, 32679, -266878310, 1376310210, 32679] [2147483647, -1, 2147483647] [-3666973055865689434, 9223372036854775807, 32679, 147483648, 1, -1678007383, -1, -698710754, 1, 245] [-9223372036854775808, 147483648, 9223372036854775807, 1, 0] [147483648, 301276449624033136, 1, 147483648, 32679, 1884200031939163268, 1, -8280470616867117869, 8887435956810483508, 0] [147483648, 147483648, -7476877933620869753, -1, 147483648, 7837717670416047590, 1, 147483648, -5902182217119133637, 7063227393143734913] [-1869482842, 0, -2, 0, -2, 1, 9, 1] [6, 1, 0, -178419190, 7, 8, 1, -1] [] [1, 0, 1082541363, 6, 0, -1, 2, 1, 0, -368314180] [81.101100000000, 79.024000000000, 0.000000000000, 1.161800000000, 0.000000000000] [40.123000000000, 40.123000000000, 14.157500000000, 40.123000000000, -1.200000000000] [1.200000000000, 300.343000000000, 300.343000000000, 1.200000000000, 0.000000000000] [-1.200000000000, 300.343000000000, 0.000000000000, -1.200000000000, 1.200000000000, 1.200000000000, 5.066600000000, 100.020000000000] [68.0851, -1.2000, -1.2000] [] [54.0463, 1.2000, 21.1092] [1.2000, 1.2000, -1.2000] \N [-1, -1, -1, 60, -1, 40, 62, -1] [1, 75, 15, 1, 40, 40, 30, 300, 1, 45] [] ["真实", "know as", "say", "-", "did"] \N ["夏娜", "-", "--", "because on time could why back really up can", "木马"] ["", "her", "", "可爱", "?", "中心站不做看课件素材创元信息开发区用在红雷资源直接", "做人有谁交流群顺驰", "with"] ["n", "time", "-", "再来", "自私", "", "going is it something out something", "as were he out okay okay hey had", "当作人体油画心事", "get will just and"] ["发过充满泡茶电脑太原先付这百年小姑娘人参瑞达", "-", ""] ["", "didn't", "?", "no me there can there is i have", "冀海潮头痛运行", "get that's we because get i him did now say", "医生", "she", "see", "b"] ["it all can some didn't here the get got", "游戏", "here", "", "from", "the now have who", "be now you here tell here", "倚天", "x", "x"] ["k", "w", "r", "x", "c"] \N ["w", "q", "n"] ["j", "e", "w", "b", "w", "s", "b", "w", "g", "c"] [] [] [2023-12-13, 2024-08-03, 2023-12-19, 2023-12-13, 2024-02-18, 2026-02-18, 2024-08-03, 2017-07-28, 2024-06-30, 2024-02-18] [2023-12-16, 2027-01-09, 2023-12-13, 2027-01-09, 2023-12-14, 2023-12-20, 2026-01-18, 2024-01-19, 2025-06-18, 2023-12-12] [2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59, 2009-02-20 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2023-12-16 00:00:00] [2014-08-12 00:00:00, 9999-12-31 00:00:00, 2024-08-03 13:08:30] [2014-08-12 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00] [2024-02-18 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30, 2016-11-22 00:00:00] [] [2023-12-11 00:00:00, 2023-12-20 00:00:00, 2024-01-31 00:00:00] [] [2023-12-19 00:00:00, 2023-12-20 00:00:00, 2002-06-03 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-13 00:00:00, 2025-02-18 00:00:00] [9999-12-31 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00] [2023-12-19 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00] [2023-01-15 08:32:59.123123, 2014-08-12 00:00:00, 2023-12-12 00:00:00] [2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-19 00:00:00, 2023-12-11 00:00:00, 2008-07-02 00:00:00, 9999-12-31 23:59:59, 2023-12-10 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00] +249 32679 2147483647 -1430649230 1 2024-01-17 2024-02-18 2024-02-18 2024-01-17 of out I'll don't ? 占用从事最后牡丹误区授权湖北 男人 这个 军港 or \N 1 300.343000 1.200000 0.000000 18.109300 2024-02-18 00:00:00 2024-07-01 00:00:00 2023-01-15 08:32:59.123000 2024-07-01 00:00:00 2024-07-01 00:00:00 2025-02-18 00:00:00 2023-12-19 00:00:00 2024-02-18 00:00:00 [1, 0, 1, 0, 1, 1, 0, 0] [1, 1, 1] [1, 1, 0, 1, 1] [1, 1, 0, 0, 1] [24, -128, 127, -128, -1, -1, 97, 127] [1, -40, 2, 1, -1, 5, 71, 7] [7, 1, 0, -1, 1] [-103, -128, 1, -119, -97, 2, -128, 0] \N [2312, -3339, 32767] [245, 245, -27430, -2570, 490] [-9753, -20961, 32767] [1876611852, -1, -729267308, -1, 0, 2147483647, 1916092746, 624214467] \N [1, 2099095685, -2147483648, 1521982620, 0, -2147483648, 2147483647, 32679, -1, -1051398637] [-2147483648, -1, -1, 0, 2147483647, 147483648, -1398926725, 32679, 147483648, 2058593669] \N [147483648, 147483648, -3718925877299780138, -4727660396368684144, -9223372036854775808] [] [4428251684128946837, -6111008679943256835, 147483648, -2920054, 1, 0, -9223372036854775808, 245, 4727025, 0] [1, -1, 0, -1, -2, -713816062, -1, -2] [-2, -2, -2] [-1135584487, -113091648, -1, -2, 1] [-2, -1, -1, -1, -2, -2, 553798124, 0, 0, 0] [300.343000000000, 100.020000000000, 0.000000000000, 23.013800000000, 100.020000000000, 1.200000000000, 40.123000000000, -1.200000000000] [0.000000000000, 300.343000000000, 100.020000000000, 27.023800000000, -1.200000000000] [1.200000000000, 1.200000000000, 0.000000000000, 100.020000000000, -1.200000000000] [30.013000000000, 21.136600000000, 5.028900000000, 40.123000000000, 0.000000000000, 300.343000000000, 0.000000000000, 100.020000000000, 1.200000000000, 48.067400000000] [40.1230, 40.0271, 35.0276, 100.0200, 0.0000, 40.1230, 1.2000, 300.3430, 6.0566, 0.0000] [73.1190, 20.1877, 40.1230, 100.0200, 0.0000, 1.2000, 87.1916, 5.1218, 56.0710, 1.2000] [] [98.0251, 47.0722, 40.1230, 100.0200, 300.3430] [40, 100, 40, 300, 0] [100, 0, 60] [28, 15, -1, 100, 10, 55, 30, 42] [1, 300, 100, 300, 43, 100, 0, -1, 1, 100] ["第二个", "m", "河南经销", "--", "what", "结识", "x", "mean had or all"] ["about", "--", "he"] [] [] \N ["go", "all here is there i", "?"] ["did if", "some could not all he why", "一套", "", "聂柯福利亚太整理快车苦楚长得阵列经济圈内"] ["小人双子我在同学家屏幕", "", "成熟", "回答问题心有", "-", "?", "", "濮阳温总客服开了相处", "", "不含"] ["p", "s", "v", "t", "x", "p", "h", "b", "h", "b"] ["g", "f", "x"] ["b", "x", "j", "h", "r"] ["q", "x", "g", "b", "m"] \N [2023-12-19, 2024-07-01, 2024-01-17, 2024-01-08, 2023-12-11, 2024-06-30, 2014-08-12, 2023-12-15] [2027-01-09, 2027-01-16, 2024-02-18, 2025-06-18, 2010-02-04] [2025-02-17, 9999-12-31, 2024-07-01, 2023-12-16, 2023-01-15, 2024-02-18, 2023-12-20, 2024-02-18, 2024-07-01, 2023-12-12] [2023-12-12 00:00:00, 9999-12-31 00:00:00, 2023-12-15 00:00:00, 2024-06-30 12:01:02, 2024-01-19 00:00:00] [2025-06-18 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00, 2025-02-17 00:00:00] [2025-02-17 00:00:00, 2024-01-31 00:00:00, 2027-01-09 00:00:00] [2024-06-30 12:01:02, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-31 00:00:00, 2023-12-11 00:00:00] \N [2027-01-16 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59] [2024-08-03 13:08:30, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2024-01-19 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2024-06-30 12:01:02.123000] [2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2023-12-12 00:00:00] [2026-01-18 00:00:00, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2026-02-18 00:00:00, 2027-01-16 00:00:00] [2024-07-01 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2025-02-17 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59.123123, 2024-07-01 00:00:00, 2011-12-04 00:00:00] [2023-12-15 00:00:00, 2023-12-09 00:00:00, 2024-01-17 00:00:00] [2023-12-20 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00] +250 -332715572 -2147483648 147483648 69368042 2024-07-01 2023-12-14 2023-12-19 2023-12-12 going about this as not it well some know yeah as of some oh ? ? y 杨宗驱动器你不表达 1 0 68.082700 59.178400 25.051500 -1.200000 2026-01-18 00:00:00 2024-01-17 00:00:00 2027-01-16 00:00:00 2024-01-08 00:00:00 9999-12-31 23:59:59 2026-01-18 00:00:00 2025-06-18 00:00:00 2023-12-17 00:00:00 [] [1, 1, 1, 1, 1] [1, 0, 0, 1, 1, 1, 1, 1] [0, 0, 0] [-94, -65, 0, -128, -6, 0, 1, -43, -46, -1] [-4, 3, 127, -1, 76] [-1, -9, -57, 1, 5, 0, -76, 86, 0, -1] [-1, 0, 112, -18, -95, -104, 0, 6] [0, 32767, -28094, 21933, 25872, 20336, -15423, 0, 245, -23245] [1125, 30174, 26691] [-21402, -3774, 6976, 21540, 1] [32767, 17169, 4896] \N \N [-995008255, -1, 32679, 2147483647, 2147483647, 1503309812, 32679, 2147483647, 1649528292, 2147483647] [] [-4506946504647987920, -1, -965787688076241882, 32679, 245] [0, 32679, 815782625, -1, 1429568729, 245, 245, 9223372036854775807, -5725659408333502614, -8963547862000303868] [147483648, 0, 1, -939511987721344924, 1] [] \N [-1, -1, -2] [] [8, -2, -1332509517, 345867491, -2, -1274319115, -2, 531275513] \N \N [40.123000000000, 0.000000000000, 66.052600000000] [0.000000000000, 1.200000000000, 40.123000000000, 100.020000000000, 300.343000000000] [100.0200, 20.1482, 65.0773, 300.3430, 1.2000, 16.0044, 300.3430, 7.0701, 72.1908, 1.2000] \N [300.3430, -1.2000, 29.0886, 73.1256, 34.0660] [1.2000, -1.2000, 90.1375] [0, 48, 15, 100, 40] [-1, 300, 44, 44, 300, -1, 30, 300, 51, 300] [0, 1, 0, 5, 30, 47, 44, 47, 300, 0] [100, 14, -1] ["-", "肛门", "not the been were"] ["are", "惊喜总代理商", "r", "--", "金辉"] ["", "回首好不好这百年连接线", "up about"] ["got", "or and really can't I'll look can't on could it", "--", "?", "伴奏", "--", "样品", "欧阳", "-", "新派打工的工作站良子保护来启用见到一下不懂面包"] ["y", "诚心", "我不知道", "需方", "I'm get up we about if i", "?", "收藏夹", "开出来"] ["-", "?", "十分"] ["而不是", "", "你不前期快车春江花月夜从你那里司军几一改跃民漯河", "yeah", "ok i", "-", "你不工作吗", "a"] ["司军几倪玲梅周哥上午机关", "ok at or no time", "不定", "when out he with don't", "-"] [] [] ["g", "h", "e", "e", "h", "b", "u", "u", "i", "d"] [] [2024-01-08, 2024-08-03, 2023-12-18, 9999-12-31, 2025-06-18, 2023-12-18, 2023-12-13, 2024-01-19, 2025-02-17, 2023-12-19] [2024-02-18, 2023-12-20, 2023-12-19] [2023-12-18, 2023-12-09, 2023-12-19, 2027-01-16, 2024-08-03, 2024-01-19, 2027-01-09, 2024-01-09, 2024-01-08, 2023-01-15] [2024-01-08, 2023-12-09, 2023-12-12] [2024-01-19 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2026-02-18 00:00:00] [2025-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2026-01-18 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-11 00:00:00] [2026-02-18 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-17 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2023-12-20 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00] [2023-12-15 00:00:00, 2001-05-26 00:00:00, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2023-12-11 00:00:00, 2007-04-05 00:00:00, 2024-01-08 00:00:00] [9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-19 00:00:00] [2023-12-17 00:00:00, 2023-12-12 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2010-04-06 00:00:00] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-15 00:00:00] [2023-12-15 00:00:00, 9999-12-31 00:00:00, 2024-01-09 00:00:00] [2019-01-05 00:00:00, 2025-02-17 00:00:00, 2023-01-15 08:32:59.123123] [2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2024-01-19 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00] +251 1710827135 0 -1883519281 2147483647 2023-12-13 2024-08-03 2025-06-18 2023-12-14 because you okay are got can well 维泽 what p 那你 really 靠前 0 1 40.123000 300.343000 300.343000 0.000000 2024-01-17 00:00:00 2027-01-09 00:00:00 2023-12-10 00:00:00 2023-12-18 00:00:00 2001-01-12 00:00:00 2023-12-15 00:00:00 2023-12-20 00:00:00 2023-12-14 00:00:00 [1, 0, 1, 0, 1, 0, 0, 1, 0, 1] [0, 1, 0, 1, 0] [1, 0, 1, 1, 1, 0, 1, 0, 0, 1] [1, 1, 1, 1, 1, 1, 1, 0, 0, 1] [-54, 1, 38, 6, 0, -125, 127, 8, 0, 0] [-121, -1, -82, 0, 1, -25, -128, -8, -26, 76] [8, 0, 7, -22, 127] [-1, 96, 42] [] [23293, -1, 1, 1, 245, -19321, -32768, 245] [1, 4285, -1, -32768, -25185, -15417, 1, -10606] [32755, -31685, -1, 21295, -15555, -32768, -32768, -24458, -32768, -28689] [] \N [1218338561, 2147483647, -1508264137, -1, 147483648] [2124679695, 1455772494, 32679] [-9223372036854775808, -9223372036854775808, 32679] [] [-7582494225091357696, -1, 9223372036854775807, 50352, 245, -2143017160097282307, -1948498985, 1692469176360494688] [] [5, -2, 0, 1, 4] [0, -1797637647, 0, 0, 0, -2, -2, 0, 1238581606, -2] [1, 2, 0] [-1, 1, -2, 1, 0, 1, 7, 1513001053, 7, 1] [40.123000000000, 2.050100000000, 300.343000000000, 36.056400000000, 1.200000000000, 300.343000000000, 0.000000000000, 100.020000000000] [] [40.123000000000, -1.200000000000, 40.123000000000, 1.200000000000, 59.079100000000, 1.200000000000, 300.343000000000, 56.135500000000, 100.020000000000, 12.038500000000] [39.096700000000, 1.200000000000, 100.020000000000, -1.200000000000, 1.200000000000, 1.200000000000, 40.123000000000, 1.200000000000, 40.123000000000, 0.000000000000] [0.0000, 40.1230, 40.1230] [] [-1.2000, 300.3430, -1.2000, 300.3430, 50.1510, 0.0000, -1.2000, 300.3430, -1.2000, 300.3430] [100.0200, 100.0200, 0.0000] [1, 40, 84, 71, 93, 0, 1, 42, 0, -1] [] [40, 100, 30, 67, 71, -1, 40, -1, 15, 66] [300, 300, 40, 43, 40, -1, 52, 0] \N ["天浩", "七匹狼有波陌生人保佑关于银盘付费图美", "-"] ["--", "", "浪费环境一部新日射七匹狼重新手册做人成交"] ["got you", "--", "", "-", "韩鸽飞顺驰兄弟手续", "小王产品事业部经理胡华威天威开机一会见谁那里经理东风路", "see got want time he because here", "早点", "", ""] ["网吧余下中小企业交货心中上不了孙瑞霞胖东来暴风影音", "", "智慧下午氛围扩展瑞丽在呀深蓝四方底下金成", "会议", "-"] ["--", "why", ""] ["mean", "?", "out", "okay going on out mean him on him something", "其实青少年钻石金牌尼奇不让", "--", "-", "柒服装刘汉", "--", "look like be"] ["that's his oh we at his the", "影子拿到不做看是广州市双子河南好句", "环境", "中原路", "who"] ["q", "r", "s", "d", "m"] ["w", "l", "e", "q", "u", "r", "p", "k"] ["o", "p", "x", "d", "x", "m", "m", "a", "d", "g"] ["r", "o", "z"] [2023-12-09, 2024-01-17, 2003-10-17] [2023-01-15, 9999-12-31, 2024-02-18, 2023-12-09, 2025-06-18, 2013-06-16, 2023-12-17, 2023-12-19, 2024-01-19, 2025-02-17] [2024-02-18, 9999-12-31, 2025-02-17] [2023-12-16, 2023-12-15, 2024-01-08] [2024-01-17 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2025-02-17 00:00:00, 2026-01-18 00:00:00, 2005-11-01 00:00:00, 2026-02-18 00:00:00] [] [2023-12-10 00:00:00, 2015-05-26 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-12 00:00:00, 2023-12-15 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2023-12-14 00:00:00] [2024-01-19 00:00:00, 9999-12-31 23:59:59, 2004-05-03 00:00:00] [2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 2014-08-12 00:00:00, 9999-12-31 00:00:00] [] [] [] \N [2023-12-16 00:00:00, 2023-12-13 00:00:00, 2018-02-21 00:00:00] [2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00] +252 -89688735 1004843924 146951937 147483648 \N 2023-12-15 9999-12-31 9999-12-31 去掉 \N 惊喜总代理商短句一夜咱用谢晓娟 \N not - 科苑 \N 1 \N 97.018300 300.343000 0.000000 \N 2024-02-18 00:00:00 2024-07-01 00:00:00 2027-01-09 00:00:00 9999-12-31 00:00:00 2024-07-01 00:00:00 2027-01-16 00:00:00 2023-12-20 00:00:00 [] [1, 1, 1, 0, 0, 0, 0, 0] [] [1, 0, 0] [119, 127, -20] [-97, -4, 1] [-128, -128, 1, 1, -128, 127, 127, 0, -64, 73] [-1, -128, 56, -128, -1, 0, -44, -128] [-23280, -1, 10520, 5766, 23457] [8473, -1, -26378, -8351, -21768, -32768, 32767, -32768, 0, 1] [0, 2018, -27722] [-32768, 32767, 1] [-1753354616, 32679, 147483648, 682872178, 1, 1, -1, 1, 2147483647, -1753702832] [-1803351138, 390397551, 1] [32679, -1520157130, 2147483647, 247296301, -695869409] [1210277274, -610834001, -76562332, -1228603789, -2147483648, -2147483648, -55639343, -1, 147483648, -1] [245, 147483648, -1567868882849899839, 1, 245, -9223372036854775808, 9223372036854775807, 1826200789, -1516721378, 3876567] [1, 631213499022296628, -1, -527543, -7479947550168306727] [4387862024489807994, -9223372036854775808, -1, 9223372036854775807, -3301744239902502320, 1541695941, 2936251, -1638541, 9223372036854775807, 3161319931587336878] [] [4, 0, -1, -1, 0, 1, -2, -2, -1, -1] [-2, -2, -1141586041, 1, 8, 1061205369, -2, 2] [0, -1, 4, 4, 1, -2, 0, -2, 1, -1] [-1, 0, 1, -2, 0] [41.035700000000, 51.117400000000, 18.007700000000] [91.105600000000, 37.071000000000, 40.123000000000, 40.123000000000, 300.343000000000, 100.020000000000, -1.200000000000, 48.042800000000, 68.138600000000, -1.200000000000] [-1.200000000000, 22.104600000000, 77.003800000000, 17.140500000000, 300.343000000000] [] [-1.2000, 40.1230, 65.0877] [40.1230, 60.0200, 38.1581, 1.2000, 1.2000, 40.1230, 41.1582, 0.0000] [] [] [40, 300, 57, 1, 300, 1, 40, 42] [300, 300, 45] [-1, 0, 33, 96, -1] [300, 1, 3, 100, 72, 300, -1, 100] ["a for want can't", "-", "弄个"] \N [] ["图形工作站中资格结婚舍得写下谢晓娟龙飞小故事成熟调货", "--", "--", "", "水利一部埋在准备帐户清楚"] ["为了", "my", "-", "l", "", "基本上注定", "年后", "--"] ["to and are", "but well we out him been say got", "?", "?", "一样无线小曹位居叁佰光通路诊断"] ["t", "-", "do i because we", "out or well tell will", "her is see all what see are in", "just", "he's you is well but he come she one yeah", "now i there"] ["see", "e", "特配机器不见性价比刻骨铭心千兆播放", "just", "then okay", "come would know some with on", "联盛回答问题下来前几天短句还得", "say think know now"] ["l", "i", "k", "l", "j", "x", "m", "a"] [] ["b", "l", "d", "t", "x"] ["j", "d", "z", "z", "p"] [2023-12-16, 2024-01-19, 2026-01-18, 2023-12-15, 2024-01-08, 2018-03-16, 9999-12-31, 2024-07-01] [2027-01-09, 2023-12-20, 2025-06-18, 2024-07-01, 2024-07-01, 2024-02-18, 2024-02-18, 2023-12-16] [] [2026-01-18, 2023-12-17, 2025-02-17] [2025-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-15 00:00:00, 2024-06-30 12:01:02, 2023-12-10 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2023-12-17 00:00:00] [2023-12-19 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2023-12-12 00:00:00] [2027-01-09 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2024-06-30 12:01:02, 2024-01-09 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2023-12-15 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2024-01-17 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2023-12-18 00:00:00] [2024-07-01 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2006-02-23 00:00:00] [2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00] [2024-07-01 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00, 2024-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2024-01-09 00:00:00, 2026-02-18 00:00:00, 2023-12-09 00:00:00] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2023-01-15 08:32:59.123000, 2027-01-09 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00] [9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2011-10-09 00:00:00] [2027-01-09 00:00:00, 2013-08-17 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2005-07-16 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00] [] [2026-02-18 00:00:00, 2024-01-09 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2023-12-12 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00, 2023-12-09 00:00:00] +253 2147483647 0 -1493784934 0 \N 2025-06-18 2023-12-11 2023-12-12 反正词条理解伟旭高就曙光研普 good 中信青青草煤气瀚海热线提示壹万失去拼搏 u 有些 how from how there will ? 1 1 300.343000 100.020000 59.012300 55.062000 2023-12-13 00:00:00 9999-12-31 23:59:59 2023-12-17 00:00:00 2023-12-12 00:00:00 2024-06-30 12:01:02.123000 2023-12-17 00:00:00 2027-01-09 00:00:00 2024-02-18 00:00:00 [0, 1, 0] [0, 1, 0, 1, 0, 1, 0, 1] [] [0, 1, 1, 1, 1, 1, 0, 1] [107, -128, 8, 73, 54] [0, 127, 3, 127, -1] [] [37, 119, 44, 23, 2, -8, 72, 127] [] [-24077, 27199, 13413, 23839, 245] [-32253, 0, 1648, 32767, 245, -13508, 1258, 245] [245, 20002, 23376, 27132, -14207, 24312, -32768, 0, -32768, 12965] [] [-832940758, 2147483647, -1] [-2147483648, 1, 32679, 1, -2147483648, 2147483647, 147483648, 1, 0, -688346899] [-689890053, 32679, -1, 1, -1, 1097876709, 2125875094, -2037904715] [1466910450, 0, 3703881830903453654, 199062344425503273, -9223372036854775808, -3091077, 147483648, 9223372036854775807, 1, 1] [147483648, 9223372036854775807, 1, 245, -8741391103054097387] [] [32679, 272941472839526043, -745431991959403901, 1, 147483648, 4654435211300612278, -3823408131434605193, 245, 32679, -985445228] [0, 0, 1, 1, 9] [0, 227590298, 1, 0, 1, -1, 1, 8, -2, -524914479] [] [4, -2, 0, 9, 1] \N [] [47.192400000000, 100.020000000000, 100.020000000000] [1.200000000000, -1.200000000000, 100.020000000000, 300.343000000000, -1.200000000000, 18.126600000000, -1.200000000000, 0.000000000000] [-1.2000, 100.0200, 93.0915, 300.3430, 0.0000, 1.2000, 1.2000, 1.2000, 13.1690, 300.3430] [40.1230, -1.2000, 40.1230, 0.0000, 40.1230, 0.0000, 40.1230, 40.1230] [58.1470, 0.0000, 0.0000, 100.0200, 40.1230, 62.0441, 300.3430, 74.1882] [40.1230, 23.0779, 45.1023] [] [100, 27, 0, 90, 40, -1, -1, 0, 49, 40] [] [73, -1, 300, 300, 44, 100, 300, 0] ["see", "你也来了", "-", "最新", "具体你来办理", "going", "新国当地叫做", "创元泥石谁控制卡", "?", "--"] ["大量贰仟不敢残品一颗", "going", "there yeah I'm he it's", "--", "?", "确实不曾制动阀方面毕业墙节点空分一鸣不完眼睁睁", "全名是你呀", "about up want and yes no the"] ["?", "know were now good like", "同事", "有种", "数据库万邦机会甘心表情"] [] [] ["投影机", "they", "?", "既然信阳内部考虑管理者选择拦不住", "k", "I'm well then go I'm i how", "--", "?", "p", "?"] ["?", "that", "?", "招待费惭愧不让一句话架子雅鹿沙原著啊借款传美迪微", "just his i oh all was because out", "生命中", "-", "卖卖按季度运费"] ["what", "about", "want", "?", "--"] ["f", "j", "q"] ["z", "g", "w"] ["z", "a", "c", "c", "p", "n", "k", "y", "q", "f"] ["u", "g", "v", "c", "e"] [2023-12-11, 2024-01-31, 2026-02-18, 2023-12-10, 2024-01-08, 2026-01-18, 2023-12-15, 2023-12-12, 2024-01-17, 2027-01-09] [2023-12-18, 2025-06-18, 2023-12-11, 9999-12-31, 2023-12-20] [2024-02-18, 2024-01-08, 2024-08-03] [2027-01-09, 9999-12-31, 2024-01-09, 2023-12-15, 2015-11-02, 2024-02-18, 2024-01-08, 2024-01-09, 2023-12-16, 2026-02-18] [2023-12-13 00:00:00, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [2024-02-18 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00] [2025-02-18 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00] [2027-01-16 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2024-01-31 00:00:00] [2013-07-05 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2023-12-16 00:00:00] [] [2014-08-12 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-08 00:00:00] [2024-06-30 12:01:02.123000, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2023-12-15 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00] [2024-01-17 00:00:00, 2023-12-13 00:00:00, 2024-07-01 00:00:00, 2023-12-15 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2024-01-17 00:00:00] [2024-01-19 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 2003-11-24 00:00:00] [2025-02-17 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00, 2023-12-19 00:00:00, 2026-01-18 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-17 00:00:00] +255 -897044792 759685125 243257026 2147483647 2011-01-10 2027-01-09 2023-12-15 2023-12-10 \N 下面 标注 me would it's well then in 踏踏实实医药 这中间谁家试过英莲 广益彭巧语一岁有误有钱老公惬意单条你不认识总代 for not 0 1 4.031000 69.066900 100.020000 300.343000 2024-01-17 00:00:00 2026-02-18 00:00:00 2023-12-11 00:00:00 2023-12-12 00:00:00 2025-02-18 00:00:00 2027-01-09 00:00:00 2024-01-09 00:00:00 2025-02-18 00:00:00 [1, 0, 0, 0, 1, 1, 0, 0] [0, 1, 0, 1, 0] [] [0, 1, 0, 0, 0, 0, 1, 1] [1, 24, -1] [6, 127, -69] [-1, -1, -128, -109, -108, 1, -1, 68] [-1, -1, 1, 127, 67, -91, 127, -71, 13, -1] \N [32767, 17673, 1, -32768, -19255, 1, 14160, 19541, 22787, -29354] [-31722, 0, 245, 0, 29460] [] [1, 2147483647, -1758212608, -531780713, 147483648, 32679, -1, 1015756671, 0, 859772867] [-2147483648, 15722142, -1266473625, 1973358349, -2147483648] [0, 1351106805, 2147483647, 147483648, 2147483647, 32679, 709064799, 1617497649, 2147483647, 32679] [] [1, -9223372036854775808, -9223372036854775808, 1, 6739888, -546948694, 147483648, 1, 1, 147483648] [-120857759, 1237383503063603881, 1] [] [-4907734496378896785, -9223372036854775808, -9223372036854775808, 5063546714928440563, 147483648, 9223372036854775807, 9223372036854775807, -9223372036854775808] [-2, 1, -1] [5, -2, 9, 0, 1] [603022603, 854506038, -2] [4, -2, -2] [38.094300000000, 40.123000000000, 5.180900000000, 70.082400000000, 37.198500000000, 300.343000000000, 80.038600000000, 100.020000000000] [39.049600000000, -1.200000000000, 1.200000000000, 40.123000000000, 300.343000000000] [1.200000000000, 40.123000000000, 1.200000000000] [] [1.2000, 40.1230, 45.0553, 97.1504, 13.1852] [40.1230, 40.1230, 100.0200, 1.1365, 1.2000, 19.0468, 300.3430, 72.0680, 100.0200, 0.0000] [40.1230, 93.0507, 40.1230, 40.1230, -1.2000, 100.0200, 0.0000, 0.0000, 40.1230, -1.2000] [100.0200, 300.3430, 100.0200] [100, 40, 40, 7, 100, 10, -1, 100] [35, 37, -1] [100, 40, -1, 66, 60, 300, 65, 300] [] ["--", "宝源", "just", "if", "they see now right him something", "what", "yes", "老同学一栏瑞恒壹万优惠还有已给不对兆文带你", "?", "驱动授权经销总代理"] ["want we when why she be", "", "", "", "?", "just", "some", "didn't", "也冲击", "同力"] ["呀库存惭愧月亮水哦也会差不多难道发生过办好神仙", "l", "to up what I'm a would like he's one", "下市场你是请客富绅屠龙记见过才好这几天祝福网络", "拜拜"] ["?", "?", "he", "--", "铭岳真正以以上的迅盘"] ["as they been yeah your", "--", "爱情汇祥创彩拓展这场暂时"] ["有机会", "of", "how", "here", "立讯", "y", "i", "you're"] ["资金", "see not hey i what he her my it's", "?", "保证三星清单云隐不够询问过", "", "-", "-", "独显", "-", "设备"] [] ["p", "c", "g"] ["y", "i", "x", "y", "o", "w", "p", "m"] [] ["d", "t", "h", "h", "g", "l", "z", "j", "j", "k"] [] [2024-07-01, 2024-07-01, 2025-06-18, 2023-12-10, 2023-12-17, 2005-11-21, 2024-07-01, 2023-12-19] [2024-01-08, 2025-06-18, 2024-01-19, 2014-08-12, 2025-06-18, 2023-12-15, 2025-02-18, 2024-01-08] [2025-02-17, 2024-02-18, 2025-02-17] [2025-06-18 00:00:00, 2023-01-15 08:32:59, 2027-01-09 00:00:00] [2023-12-20 00:00:00, 2027-01-16 00:00:00, 2025-02-17 00:00:00] [2023-12-13 00:00:00, 2023-12-12 00:00:00, 2024-01-08 00:00:00] [2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-10 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2025-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-15 00:00:00] [2024-02-18 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2024-01-19 00:00:00] [2023-12-09 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00] [2025-06-18 00:00:00, 2014-08-12 00:00:00, 2023-12-14 00:00:00] [2023-12-11 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00] [] [] +256 -2097925081 32679 -964393672 2147483647 2023-12-11 2023-12-19 2025-02-17 2010-01-03 why that 屏蔽 \N ? because 张总 1 1 100.020000 1.200000 48.081800 300.343000 2024-01-31 00:00:00 2003-05-27 00:00:00 2027-01-16 00:00:00 2023-12-14 00:00:00 2025-06-18 00:00:00 2023-12-15 00:00:00 2024-07-01 00:00:00 2025-02-18 00:00:00 [0, 0, 0, 1, 1, 0, 0, 0] [0, 1, 1, 0, 1, 1, 1, 0] [0, 0, 0, 0, 0] [1, 0, 1, 0, 1, 1, 0, 1, 1, 0] [-128, -128, 127] [-97, 0, 0, -128, 1, 127, 1, 1, 0, 0] [1, -128, 0, 6, 1] [0, 1, -28, -1, -128, 3, 49, -1, -128, -128] [7059, 0, 9345, 12737, -16327, -21287, -22986, -24724] [-6614, 32767, 31129] [-3102, 32277, -32768] [] [-157927737, 1304485099, 1344150782, -2147483648, -1083837804, 1352936837, 32679, 780976943] [-431918274, -1519373626, 1482075378, 1213332685, 0, 147483648, -1, 290896798] [0, 1, -2147483648, 1, 951600091, 0, 0, 2147483647, -851452564, 147483648] [1570033657, -1264307744, 1, 0, 2147483647, -2147483648, -823725643, 201359779] \N [] [-2641572904652301668, -948694480, -7540101228250953781, 3166014, 245, -9223372036854775808, -1, -487605566, 147483648, 147483648] [-2538970, -3451366, 32679, 1, 750737417] [] \N [-1051165068, 9, -754867723, 7, -1618516390] [-1, -725156880, -2, 1, -1, 1, -2059715160, 1] [100.020000000000, 89.119200000000, 300.343000000000, 300.343000000000, 300.343000000000] [300.343000000000, 51.082800000000, 0.000000000000, 300.343000000000, 50.077400000000] [78.023800000000, 100.048400000000, 0.000000000000, 97.181000000000, 300.343000000000] [] [85.1212, 19.0398, 100.0200, -1.2000, 100.0200, 77.0772, -1.2000, 1.2000, -1.2000, 300.3430] [0.0000, 40.1230, 2.1210, 300.3430, 100.0200, 100.0200, 0.0000, 300.3430] [1.2000, 69.1474, 1.2000, 1.2000, 300.3430, 40.1230, 300.3430, 40.1230, 100.0200, -1.2000] [] [34, 100, 1, 40, 0, -1, 1, 15, 0, 1] [15, 62, 0, 40, 44] [] [100, 54, 1, 40, 1] ["?", "", "k", "卢海有些才是我给你打电话四方", "顺风"] ["", "刘海", "we they don't for him now", "--", "询问过大姑制胜我在郑州不做小曹充满还好吧", "", "didn't", "know she I'm her"] ["this", "l", "用得上", "?", ""] ["who", "", "不行限价中欧", "ok time didn't oh as want what", "", "?", "--", "天普", "", "?"] ["元月", "--", "-"] \N ["休息打字纸业", "out as", "有点忙", "s", "?", "l", "--", "?"] ["很想老刘金牌经销总代理欢迎欢迎政务不太", "?", "不小嵩县柒服装柒服装这中间带来第八深圳", "费心订货政府姓名残品故事母亲暴风影音下游昨天", "--", "q", "want out go or out did", "", "-", "?"] ["m", "q", "r", "c", "o", "r", "i", "b", "u", "a"] ["u", "y", "b", "o", "w"] ["f", "y", "g", "l", "g", "w", "n", "o", "n", "z"] [] \N \N [] [2025-02-17, 2027-01-09, 2024-01-08, 2023-12-19, 2024-02-18] [2023-12-17 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2014-03-15 00:00:00, 2025-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-06-30 12:01:02] [2025-06-18 00:00:00, 9999-12-31 00:00:00, 2027-01-09 00:00:00, 2024-06-30 12:01:02, 2023-12-09 00:00:00] [2023-12-14 00:00:00, 2025-02-17 00:00:00, 2023-12-14 00:00:00, 2014-08-17 00:00:00, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2027-01-09 00:00:00, 2004-06-05 00:00:00, 9999-12-31 00:00:00, 2023-01-15 08:32:59] [] [2024-01-17 00:00:00, 2023-12-11 00:00:00, 2024-08-03 13:08:30] [2023-12-09 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] [] [2024-07-01 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00] [2023-01-15 08:32:59.123123, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [2014-08-12 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2010-09-06 00:00:00] [] +257 1512315391 32679 648569022 2147483647 2024-02-18 2023-12-09 2023-12-15 2024-02-18 how it his on are know I'll okay I'm in got 用眼清单魏军老人家那么大尼奇 小孩 ? -- 0 1 53.021600 40.123000 300.343000 0.000000 2025-06-18 00:00:00 2023-12-09 00:00:00 2024-01-08 00:00:00 2027-01-16 00:00:00 2027-01-09 00:00:00 2023-12-16 00:00:00 9999-12-31 00:00:00 2026-02-18 00:00:00 [1, 1, 1, 0, 1, 0, 0, 0] \N [0, 0, 1] [0, 0, 1, 0, 0] [-128, 1, 1] [0, -37, -128, 49, 7, 112, -48, -58] [127, 0, 0, 29, -75, 1, 127, 100] [1, 75, 79, -75, -10] [9003, -25974, 13139, -1, 21330, -29300, 14412, 1, -30064, 11891] [1, -15247, -6455] [4715, 15383, -32768, -29606, 28960] [32767, 32767, 8354] [-2147483648, 1, 147483648, 1753770742, -1510278476, -397471363, 147483648, 32679] [-1619231146, 2147483647, 1122592672, -1739044211, -2147483648, -1865827613, 1, -1224236596] [1527483815, 0, -1] [327573071, 32679, 1, -602925537, 0, -2147483648, -909288092, -1382936177] [-8677338502922642225, 4634601, 4344741056654051320, 147483648, 147483648, -263804760533678572, -9223372036854775808, -7981194292686674694] \N [-1, 9223372036854775807, 569862868, 0, 2079186114907172686] [] \N \N [] [-1, 1, 5, 0, 1, 1, 7, -2] [23.079000000000, 20.081800000000, -1.200000000000, 300.343000000000, -1.200000000000, 38.048800000000, 40.123000000000, -1.200000000000] [4.038400000000, -1.200000000000, 100.020000000000, 1.200000000000, 100.020000000000, 40.123000000000, 300.343000000000, 100.020000000000, 40.123000000000, 89.011200000000] [82.003900000000, 0.000000000000, 88.002400000000, -1.200000000000, -1.200000000000, 300.343000000000, 32.036200000000, 0.000000000000, 1.200000000000, 82.029600000000] [] [100.0200, 92.1165, 94.1194, 40.1230, 91.0662] [40.1230, 71.0726, 40.1230, 100.0200, 300.3430, 0.0000, 1.2000, 54.1732, 300.3430, -1.2000] [300.3430, 63.0846, 40.0811, 53.1583, 68.0424] [] [] [60, 1, 0, 1, 100, -1, 30, 300, 0, 40] [] [0, 300, 100, 0, 300] ["?", "--", "look", "", "上线", "?", "-", "还有", "i who the good up who right be hey he", "字库文件"] ["?", "授权经销总代理", "过节费提成点方法后颈华对了降低中环"] ["电影格瑞", "back get think what can is with see didn't as", "don't what something okay it's what", "o", "--", "q", "did get", "耽误", "--", "礼数"] ["his but why really", "肯自己", "?"] ["?", "原著啊", "这百年天气"] ["又是一千多选择写下不传无论下个", "some", "--"] ["", "-", "使用见到称呼详谈这次去网讯双千兆经典电视业绩"] ["刘汉网景速度又去总经理客户端那你", "--", "金牌总代理", "t", "c", "", "莱克特宝莱怀念", "this"] ["l", "f", "j", "j", "i", "u", "d", "i", "f", "a"] ["l", "h", "d", "z", "n"] ["v", "i", "q", "i", "w", "e", "y", "n", "f", "w"] ["e", "n", "t", "h", "s", "d", "f", "g"] [] [2027-01-16, 2024-07-01, 2023-12-14, 2023-12-13, 9999-12-31] [2023-12-11, 2026-01-18, 2027-01-09, 2025-06-18, 2014-08-12, 2023-12-10, 2025-02-17, 2024-02-18] [] [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] \N [2026-01-18 00:00:00, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2008-06-01 00:00:00, 2024-01-08 00:00:00, 2025-02-17 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-12 00:00:00, 2024-07-01 00:00:00] [2023-12-19 00:00:00, 2025-06-18 00:00:00, 2026-01-18 00:00:00, 2023-12-12 00:00:00, 2023-12-09 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 2023-12-12 00:00:00, 9999-12-31 23:59:59, 2023-12-20 00:00:00] [2025-02-18 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-10 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00] [2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-01-17 00:00:00] [] [] [2023-12-15 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00] [2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2014-12-13 00:00:00, 2026-01-18 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00] [2012-05-24 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] +258 -2127833002 -2147483648 2147483647 -1 2025-06-18 9999-12-31 9999-12-31 9999-12-31 -- w d don't up 深爱档次我不会玩第二款性价比入职而来 you -- oh time come her okay out had yes that's 1 1 300.343000 \N -1.200000 -1.200000 2024-01-17 00:00:00 2024-07-01 00:00:00 2024-02-18 00:00:00 2023-12-20 00:00:00 2024-01-09 00:00:00 2023-12-20 00:00:00 2023-01-15 08:32:59.123123 2024-07-01 00:00:00 [0, 1, 0, 1, 1] [0, 0, 0, 0, 0, 0, 1, 0, 1, 0] [0, 1, 0, 0, 0, 0, 0, 0, 1, 0] [0, 0, 1, 0, 1] [-128, 73, -128, 122, 123] [0, -1, 127, 2, 7, -1, 0, -127] [-128, 1, 7, 2, -128] [-128, 127, -128, -99, -1, -1, 1, 5] [245, -18672, -4687] [0, 1, 245, -23060, -4350, 3505, 15737, -1, 25288, 1] [] [] [-273910314, -2147483648, 147483648, 32679, -1293985283] [-1, -2147483648, -1, -443373744, 526776478, 147483648, -1, -562911294, 1686573374, -416288228] [1, 1, -2147483648, -1, -214332355, 343977608, 1103226984, -2025627588, 1748402351, 0] [2147483647, 1, 332873532, 147483648, 0] [] [-9223372036854775808, 5306847605070613958, -7028820, -1, 7913142716535642053, 9223372036854775807, -9223372036854775808, 1] [32679, 0, 245, 9223372036854775807, -9223372036854775808, -7599826123642706576, -9223372036854775808, 0] [780956492953752773, 1, -9223372036854775808, -313696441, 0, 1327890624172568578, 6313026915857393760, 9223372036854775807, 147483648, 0] [1, -1, -664798829, -2, -1, -1, -2, 1, -996435957, 2095431376] [-1, 3, 2, 6, 1, -1, 2, -2] [2, 0, -2, -1, -2, -1, 0, 0] [] [0.000000000000, 30.070200000000, 12.167400000000] [40.123000000000, 0.000000000000, 300.343000000000, 36.073900000000, 40.123000000000] [0.000000000000, 52.015100000000, -1.200000000000, 33.054400000000, 93.063300000000] [0.000000000000, 100.020000000000, 300.343000000000, 42.146000000000, 83.195800000000, 1.200000000000, 100.020000000000, 48.069500000000, 18.061100000000, 8.095200000000] [100.0200, 0.0000, 0.0000, 66.0822, 92.0083, -1.2000, 36.0708, 300.3430, -1.2000, -1.2000] \N [300.3430, 40.1230, 100.0200, 0.0000, 0.0000] [38.0684, 1.2000, 40.1230, 100.0200, 66.1017] [-1, -1, 40, 58, 0, 79, 39, 40] [0, 0, 300, 64, 70, 75, -1, 73] [12, 0, 40, 1, 89] [-1, 40, 300, 55, 55, 300, 1, 300] ["--", "一小", "-", "n", "?"] \N [] ["拦不住有点不好", "--", "--"] ["yes", "金牌", "老人家山东礼貌告诉液晶不低于思科有带你企业版珍惜", "when or see really about don't at", "?"] ["", "?", "两者有单子这场说不中创"] ["过程", "景峰做成赵洪应你问没有到货做系统是你呀说不出不过呀", "for to out didn't you're yes me we me it's"] ["心事海涛艾蕊徘徊", "-", "", "?", "his back well this or of at be on been", "i not there something", "报表", "协奏曲"] ["b", "l", "h"] ["z", "z", "b", "s", "o", "y", "j", "g"] ["i", "q", "p", "o", "t"] ["i", "a", "a"] [2023-12-10, 2024-01-09, 2023-12-16, 2023-12-15, 2023-12-11, 2023-12-16, 2024-07-01, 2024-02-18, 2024-08-03, 2014-08-12] [2023-12-19, 2026-02-18, 2023-12-18, 2023-12-09, 2023-12-13, 2024-01-08, 2024-01-19, 2024-07-01] [] [2023-01-15, 9999-12-31, 2024-01-08] [2023-12-13 00:00:00, 2008-11-09 00:00:00, 2024-01-09 00:00:00] [] [2026-01-18 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00] [2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00, 2023-12-14 00:00:00, 2023-12-15 00:00:00] \N [2023-12-19 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00, 2023-12-15 00:00:00] [2024-01-17 00:00:00, 2024-01-19 00:00:00, 2024-01-17 00:00:00, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 9999-12-31 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00] [2023-12-10 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00, 2023-12-15 00:00:00, 2023-12-10 00:00:00] [2027-01-09 00:00:00, 2023-12-18 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-10 00:00:00] [9999-12-31 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 9999-12-31 23:59:59, 2024-01-08 00:00:00] [2024-02-18 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00] [2023-01-15 08:32:59.123123, 2018-11-19 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2025-02-17 00:00:00] +259 -1022769483 32679 -2147483648 970674288 2019-11-08 2014-08-12 2024-01-08 2025-02-17 so that's who him go who got on her o \N 天华 b going they don't was -- \N 0 100.020000 100.020000 40.123000 -1.200000 2024-02-18 00:00:00 2023-12-20 00:00:00 2023-12-10 00:00:00 2023-12-15 00:00:00 2024-06-30 12:01:02.123000 2024-01-08 00:00:00 9999-12-31 23:59:59 2024-02-18 00:00:00 [] \N [1, 0, 1, 1, 1] [1, 0, 0] [-1, 125, -39] [99, -117, 0, -20, 72, 16, 7, 127, -116, 127] [2, 9, -115, 127, -99] [] [-32768, 18933, 31525, 19863, 27327] [-25837, 245, 245, 0, -24677, 30137, 245, 28985] [-12124, 12206, -1, 31496, 245, -1494, 245, 0, 0, -30978] [] [-1882604049, -977255936, 1, -959525974, 2118690244] [-1588954800, -2147483648, 642006463, -170746637, 32679] [-1, 0, -974574906] [-2147483648, 2147483647, 1151929841] [-1239457317, -4068810682140152199, 32679, 1, -2064137167, 32679, 245, 9223372036854775807] [9223372036854775807, 147483648, 846308062, -1, 5876615967769170245] [] [-4071836506592176264, 245, 1, 245, 2483657998470379344] \N [-1, 359208229, -1, 2, 4] [8, -1489856126, -1, 0, -1] [-1301267656, 832554796, -2, 1, 0, 575730644, 0, -1] [100.020000000000, 40.123000000000, -1.200000000000] [0.000000000000, -1.200000000000, 300.343000000000] [-1.200000000000, 100.020000000000, 64.100000000000, 100.020000000000, 1.200000000000, 87.019800000000, 82.176100000000, 1.200000000000] [53.049200000000, 300.343000000000, 40.123000000000, 40.123000000000, 0.000000000000] \N [95.1530, 100.0200, 300.3430] [] [100.0200, 65.0648, 300.3430, 0.0000, -1.2000, -1.2000, 300.3430, -1.2000] [67, 53, 300] [40, 40, 100, 1, 47, 1, 100, 1] [] [75, 0, -1, 40, 40, 0, 300, 61, 47, 300] ["out what that's time been come is his from a", "", "-", "but", "?"] ["going could what from", "think when up when but that something have", "something or I'll no we they what really from got"] ["", "安排一周", "库里", "女朋友住宿费含有半天班了解", "-", "还跟他", "低配几万无锡最重要推推网站谢谢考勤", "p"] [] ["was", "him go be as we go", "m", "核算", "陇海路女孩子朱晓明安全儿孙们电子连续区分广州市平时", "-", "天翔", "欢迎期限里面多少钱瑞星", "your who he's here because back that back", "well it's can mean who a look something that"] ["-", "武汉", "had yeah you're in yes we got one"] ["", "?", "been your he why had you're will his", "at me were see see up he's I'm her", "-", "", "that know don't from if we", "实际上遥控器智者业务伟旭正信志彬"] ["your", "", "隆康开拓爸妈制度之作欣慰来说路上", "", "机箱", "b", "?", "?"] ["r", "j", "n"] ["a", "o", "s", "x", "a", "o", "d", "k", "h", "k"] ["m", "e", "o", "n", "r"] ["b", "e", "j", "w", "j", "u", "o", "o", "x", "a"] [2024-02-18, 2023-12-11, 2023-12-09, 9999-12-31, 2014-08-12, 2023-12-12, 2027-01-16, 2023-12-16] [2024-02-18, 2023-12-16, 2023-12-19, 2024-02-18, 2024-01-08, 2023-12-14, 2023-12-17, 2023-12-10, 2023-12-11, 2023-12-18] [9999-12-31, 2023-12-15, 2025-06-18, 2023-12-18, 2025-06-18, 2024-06-30, 2024-07-01, 2024-01-31, 2023-12-14, 2024-02-18] [2025-06-18, 2027-01-16, 2023-12-13, 2025-06-18, 2025-02-17, 2024-02-18, 2027-01-16, 2024-02-18] [2023-12-09 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2025-02-17 00:00:00, 2023-12-15 00:00:00, 2023-12-20 00:00:00, 2023-12-19 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2023-01-15 08:32:59] [] [2023-12-11 00:00:00, 2026-02-18 00:00:00, 2023-12-20 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00] [2023-12-18 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2024-01-19 00:00:00] [2023-01-15 08:32:59.123000, 9999-12-31 00:00:00, 2026-01-18 00:00:00, 2026-02-18 00:00:00, 2005-11-22 00:00:00, 2024-01-19 00:00:00, 2023-12-15 00:00:00, 2023-12-15 00:00:00] [] [2014-08-12 00:00:00, 2023-12-09 00:00:00, 2023-12-17 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 9999-12-31 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-17 00:00:00, 2023-12-18 00:00:00] [] [] [2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2009-01-25 00:00:00, 2025-06-18 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00] +260 681183487 1562379852 2103648097 32679 2023-12-12 2023-12-12 2024-06-30 2023-12-16 中原合同区别白晶计划张总没时间烟厂第八一句话 万兴 -- he's ? 等会 from there got of 0 0 100.020000 40.123000 100.020000 0.000000 2024-01-17 00:00:00 2023-12-19 00:00:00 2023-01-15 08:32:59.123000 2025-02-18 00:00:00 2026-02-18 00:00:00 2024-01-31 00:00:00 2024-01-08 00:00:00 2023-12-15 00:00:00 [1, 0, 1, 0, 0] [0, 1, 1] [0, 0, 1, 1, 0, 1, 1, 1] [] \N [1, -128, -1, -128, -128] [] [] [] [0, 245, -4963, 32767, -30788, -12818, -16499, -32768, 24951, -19193] [] [] \N [147483648, 32679, -1, -2147483648, 32679, 0, -555417718, 2147483647] [1196434454, -809778939, -1, 118660550, -1687090546] [2147483647, 2147483647, 147483648, 1988647916, -2147483648, -1, 0, 1501455467, 1, 1142033960] [6026623, 245, 32679, -1, -9223372036854775808, 32679, 0, -3533820] [-1243928455, 9223372036854775807, 0] [4147984, 245, 245, 9223372036854775807, -8179129580797867958, 2030433233126383970, 9223372036854775807, 9223372036854775807, 1, -1803261] [-1405243152026790301, 5529849952927347431, -9223372036854775808, 245, -9223372036854775808, -1, 4973762065495267963, -8487675480153974833, -1391395940, 5302736398911276802] [-1, 1, 1] [-1, 9, -1, -1, 0] [1, 0, 0, 0, -2, -1, 0, 0] [1, 0, 1, 0, -1, 1, 1, -1, 0, 1] [100.020000000000, 0.000000000000, 1.200000000000] [44.038400000000, 300.343000000000, 100.020000000000, 40.123000000000, 40.123000000000, 12.013700000000, 80.073100000000, 31.091400000000, 1.135800000000, 2.018600000000] [1.200000000000, 76.086700000000, 25.180200000000, 1.200000000000, -1.200000000000, 300.343000000000, 100.020000000000, 1.200000000000] [17.117100000000, 0.000000000000, 100.020000000000, 0.000000000000, 90.131400000000] [43.1843, 60.1171, 100.0200, 40.1230, -1.2000] [300.3430, 40.1230, 17.0007] [] [100.0200, 40.1230, -1.2000] [-1, 24, 100, 66, 40, 1, 0, -1] [300, 300, 1, 40, 59, 0, 40, 35, 100, 300] [0, 0, 0] [1, 22, 99, 300, 62, -1, -1, 65, 1, 300] ["", "mean how he about", "通过云鹏培新我刚回来呀慢慢标底", "o", "兼容性"] ["in", "是是非非注意", "going", "那款", "南阳一点很难启用我在政治哦重新我不知道你是谁去年班车", "加点要不然谁家有货附件靠前", "see", "--"] ["相见保密注册表定金结婚你好呀天地为你叫做", "up her one have it's", "加点金海软件园彩虹彭巧语喜爱热卖这边一辈子一帆", "", "the a she one can some not he", "数据班车遥控器中欧甘心双千兆", "--", "-"] ["with if just he when go that's", "", "--", "if okay going go for from her there", "--"] ["was him that on they he yeah are he's no", "电话本", "in your oh will", "购买初次五万", "mean it i", "-", "at up as will he's my could come could", "like", "名杂系统集成台式机双千兆整合长期独立", "r"] ["or", "didn't", "what could who about are is", "没沈晓海", "?"] ["-", "my", "had"] ["who about going", "拜托联通意义上不对外百元整", "go", "this", "四核", "", "-", "又给网站现货女孩寂寞伟旭王庄乡"] ["u", "b", "q"] ["s", "i", "t"] ["i", "t", "g", "s", "i", "c", "n", "l", "e", "a"] ["n", "j", "o", "t", "s", "w", "l", "q"] [2024-01-09, 2026-01-18, 2024-07-01] \N [] [] [2023-12-17 00:00:00, 2027-01-16 00:00:00, 2025-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00, 2023-12-18 00:00:00, 2025-02-18 00:00:00] \N [2027-01-16 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2008-11-13 00:00:00, 2026-02-18 00:00:00] [] [] [] [2024-01-31 00:00:00, 2013-04-02 00:00:00, 2023-12-09 00:00:00, 2023-01-15 08:32:59.123000, 2005-03-07 00:00:00] [2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2024-01-17 00:00:00] [2025-02-17 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00] [2011-09-24 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00] [2023-12-09 00:00:00, 2027-01-09 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2023-12-15 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00] [] +261 \N -2147483648 32679 2147483647 2023-12-14 2023-12-17 2023-12-15 2024-02-18 一次 -- on got not at well of out do get really when like her can't he's it ? ? because 1 0 8.005600 40.123000 86.123700 -1.200000 2023-12-13 00:00:00 2023-12-19 00:00:00 2024-01-09 00:00:00 2024-08-03 13:08:30 9999-12-31 23:59:59 2024-01-31 00:00:00 2026-02-18 00:00:00 2023-12-15 00:00:00 [1, 0, 0, 1, 1] [1, 1, 1, 1, 0] [] [0, 0, 0, 0, 1, 1, 0, 0] [-128, 3, -93, 0, 1, 6, -128, -1] [1, 127, 127, 9, 5, -17, -127, 116, -128, 0] [] [57, 127, 1, -1, 127, -40, 127, 112, 117, 8] [-9093, 32767, 245] [11950, -1, 245, -13298, -8932, 32767, -32768, -1, -1, 245] [-19511, 32767, 28980, 28855, -32768] [27480, 245, 245, 18626, 20708, -1, 27282, 4528, 26834, -15033] [32679, -1641134393, -2083594196, 140834699, 763723081] [2147483647, 1, 32679, 147483648, 0] [] [0, 1, 1, 2147483647, 1837153639, 0, -989167780, 0] [9223372036854775807, 245, 794163607160092181, 7578726, 147483648, 539098718969076280, -873699261, -1, 9223372036854775807, -7777151766720840427] [2213915, 0, -8888079300206657557] [] [147483648, 1, -9223372036854775808, 535978652399979273, -9223372036854775808] \N \N [-1, 1, 0, -1, -495600061, 0, -2, -2] [-2, 1, -1, 1, 3, 1, 0, 0] [34.011800000000, 100.020000000000, 88.179000000000] [0.000000000000, 63.174100000000, 300.343000000000, 0.000000000000, -1.200000000000] [300.343000000000, -1.200000000000, 15.101800000000] [12.138900000000, 0.000000000000, 300.343000000000, 100.020000000000, 14.012500000000, 20.123800000000, 71.112500000000, 100.020000000000] [300.3430, 26.1358, 40.1230, 1.2000, 86.1371, -1.2000, 76.0219, 26.1868, 93.1684, 100.0200] [81.1374, 32.1266, 300.3430, 300.3430, 0.0416, 0.0000, 100.0200, 22.1483, 100.0200, 300.3430] [] [1.2000, -1.2000, -1.2000] [] [] [22, 66, 100] [29, 84, -1, 96, 300, 23, 76, 14, 300, 100] ["--", "j", "really"] [] [] ["been don't not go good it's", "-", "j", "", "牡丹还好吧其他注定四天", "oh it's not your want come", "普通印象太不濮阳又给小曹", "could one why yes because could"] [] ["w", "his", "-", "b", "--"] ["--", "动手脚", "--", "okay go got a know and of from on out", "good", "oh if yeah her had want can", "c", "广场"] ["--", "祝福就会看过发到聊聊金冠多大惊喜总代理商西开发培新", "and", "from", "新买"] ["o", "c", "l", "u", "k", "l", "l", "x"] ["r", "e", "p", "u", "k", "r", "r", "p", "l", "w"] ["a", "w", "o", "n", "g"] [] [2026-02-18, 2023-12-13, 2027-01-09, 2026-01-18, 2024-06-30] [2023-12-11, 2025-02-18, 2024-08-03, 2025-02-17, 2023-01-15, 2023-12-20, 9999-12-31, 2024-01-17, 2023-12-12, 2023-12-09] [2024-01-17, 2024-08-03, 2025-02-17, 9999-12-31, 2024-02-18, 2025-06-18, 2023-12-20, 2003-06-09, 2026-01-18, 2024-06-30] [2024-01-19, 2024-01-19, 9999-12-31, 2024-01-31, 2023-12-15, 2027-01-16, 2024-01-31, 2025-06-18, 2023-12-19, 2024-01-09] \N [2026-02-18 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59] [2025-02-17 00:00:00, 2023-12-12 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2024-08-03 13:08:30, 2024-01-19 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00] [2024-01-17 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2024-08-03 13:08:30] [2024-01-17 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00] [] [2023-12-10 00:00:00, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59] [2023-01-15 08:32:59.123123, 2023-12-11 00:00:00, 2003-04-21 00:00:00, 2023-12-19 00:00:00, 2023-12-10 00:00:00] [2024-08-03 13:08:30, 2024-06-30 12:01:02.123000, 2023-12-12 00:00:00] [] [2014-08-12 00:00:00, 2024-08-03 13:08:30, 2024-01-17 00:00:00, 2023-12-15 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] +262 2147483647 1480107279 147483648 147483648 2023-12-16 2023-12-17 2026-01-18 2026-02-18 I'm how you're tell tell want right why 困扰着 who 号发给 tell you're 头痛 \N 0 64.031400 40.123000 30.172600 39.029600 9999-12-31 00:00:00 2026-02-18 00:00:00 2023-12-16 00:00:00 2027-01-16 00:00:00 2027-01-16 00:00:00 2012-06-19 00:00:00 2024-01-09 00:00:00 2023-12-17 00:00:00 [0, 0, 0] [1, 1, 0] [1, 1, 1, 0, 0, 0, 0, 0, 1, 0] [0, 1, 1, 0, 0, 1, 0, 1] [1, 6, 0, -89, -1] \N [] [] [1, 26663, -5159] \N [32767, 32767, -32768, -1459, -26690, 3743, 0, -1] [13604, 1, -1, -9166, 1] [] [1, 0, 147483648, -1474029886, 2147483647] [-925546339, -2147483648, 660701177] [1, -1820242790, 1150218477, 1615597353, 1827635574, 147483648, 780273976, 1, -2147483648, 849941648] [] [147483648, 1, -7348044, 1454782818, 245] [] [-3082266312180259823, 0, 147483648] [1, 1, -2, 8, 1, 6, 2, 5] [9, 1, -2] [-2, 0, 0, -1, 3] [] [30.021600000000, 1.009800000000, 35.071800000000, -1.200000000000, 35.083000000000, 0.000000000000, 10.163200000000, 12.199100000000] [7.038100000000, 100.020000000000, 1.200000000000, 5.012400000000, 94.156200000000, 0.000000000000, 0.000000000000, 40.123000000000] [-1.200000000000, 0.000000000000, 40.123000000000, 32.060800000000, 1.200000000000] [40.123000000000, -1.200000000000, 300.343000000000, 1.200000000000, 300.343000000000] \N [40.1230, 100.0200, 30.1966, 100.0200, 0.0000] [1.2000, 100.0200, 50.1609, -1.2000, 300.3430, 93.0293, 40.1230, -1.2000, 70.1442, 300.3430] [40.1230, 40.1230, 40.1230] [] [-1, -1, 1] [27, -1, 0, 68, 300] [100, 18, 58, 40, 1, -1, 68, 300] ["圈内创恒神秘重新发分区多多元整刘海", "really well with in really if he", "?", "", "one", "if can't and there look so or time", "刀片服务器他娘的", "not"] ["一种", "it something I'll", ""] ["oh", "订单银河", "i", "?", "轻松"] ["-", "-", "there got a as from with"] [] [] [] ["there", "", "it", "一套", "贰台"] ["m", "f", "x"] \N ["m", "o", "g", "o", "p"] ["n", "h", "r", "n", "o", "v", "y", "m"] \N [2026-02-18, 2023-12-20, 2023-12-16, 2023-12-15, 2023-12-10, 2014-08-12, 2023-12-16, 2024-07-01] [2024-07-01, 2024-02-18, 2024-07-01, 2024-07-01, 2024-01-31, 2024-01-19, 9999-12-31, 2024-02-18] [2014-08-12, 2023-12-17, 9999-12-31] [2023-12-13 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00] \N [] [2023-12-12 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [2024-01-09 00:00:00, 2027-01-09 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-17 00:00:00, 2024-02-18 00:00:00] [2015-05-12 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-09 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2016-09-24 00:00:00, 2023-12-16 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00] [2023-12-09 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-15 00:00:00, 2024-01-17 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59.123000, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00] [] \N [2026-01-18 00:00:00, 2023-12-16 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00] [2024-01-17 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2014-08-12 00:00:00] +263 -1551279082 1 -1 32679 2026-01-18 2026-02-18 2023-12-11 2023-12-13 剑派那么大而不是凄楚祝你好充满金达家庭谈好疑问 中原 黄黎明幸福带来报账机柜张婧俐怎么样优惠 so g 只要 her to but \N 1 75.084500 10.125800 46.084100 99.076800 2023-12-19 00:00:00 2024-01-09 00:00:00 9999-12-31 23:59:59 2023-12-12 00:00:00 2006-07-26 00:00:00 2023-12-12 00:00:00 2023-12-18 00:00:00 2025-06-18 00:00:00 [0, 1, 0, 1, 0, 1, 0, 0] [0, 0, 1] [1, 0, 0, 0, 0, 0, 0, 1] [1, 1, 0, 1, 0, 0, 1, 1, 0, 1] [] [-1, -20, 3, -128, -97] [9, 12, 8] [0, 61, -4, -1, 0] \N [-24736, -24494, 32767, -7898, -10560, -7663, 0, 245] [32767, 0, -1, -17936, 1] [-24341, 1, 1, 0, 7908] [1, -1, 2147483647, -1, -1170309523, -2147483648, 195342390, -994239438, 32679, 712816594] \N [-2145114170, -1131734174, 1229426002, 1204085173, 147483648] [32679, 147483648, -2147483648] \N [0, -5565108, 147483648] [] [1, 592293, -1] [9, 1, -2016242085, -1, -2, -1, -2, 5, -1541275308, 1] [-1, 9, 1, -2, 1864770560, 0, 214856623, 1, 0, 5] [-1, -1, 0, 1, -2, -2, 921775874, -1, -2, 8] [-1, 2, 4, 1676439624, 4] [] [32.045900000000, 1.200000000000, 78.198700000000, -1.200000000000, 100.020000000000] [88.058100000000, 0.000000000000, 40.123000000000] [40.123000000000, 300.343000000000, 42.198100000000, 1.126300000000, 64.010700000000, 100.020000000000, 100.020000000000, 1.200000000000, -1.200000000000, 5.168500000000] \N \N [] [54.0492, 40.1230, 40.1230, 100.0200, -1.2000, 5.1710, 54.1024, 1.2000, 69.1802, 40.1230] [46, 300, 1] \N [48, 100, 0] [40, 78, 0, 100, 59] \N ["I'll", "--", "i", "经常", "发吧他娘的询价考虑王岩春去掉", "先锋", "would", "why", "were", ""] ["", "c", "n", "-", "?", "c", "time up in that back", "发展世家", "--", "because back I'm mean have my you you're tell so"] ["your", "--", "女孩子热情劲实际客户大山大兵看法"] ["r", "know", "尽管煤炭", "at", "关于", "个人", "--", "?"] [] ["--", "k", "to the", "me right here there are get if one you're", "something was back", "-", "漫长梦幻水族大姐上线不开郑大", "--"] ["-", "is", "--", "天讯", "", "", "淀粉", "-"] ["f", "c", "j", "v", "t", "m", "s", "z"] ["s", "t", "s", "s", "z", "s", "q", "g", "a", "i"] ["m", "v", "m", "x", "v", "o", "v", "s"] ["b", "s", "m", "y", "t", "a", "t", "j", "o", "b"] [2023-12-12, 2023-12-11, 2026-01-18, 2023-12-15, 2025-06-18, 2025-06-18, 2023-12-09, 2024-06-30] [2023-12-19, 2025-06-18, 2023-12-18, 9999-12-31, 2025-02-18, 2024-06-30, 2025-06-18, 2023-12-18] [2025-06-18, 2023-12-15, 2024-02-18, 2023-12-16, 2027-01-09, 2024-01-31, 2024-01-08, 2024-02-18, 2023-12-20, 2027-01-16] [2024-01-19, 2024-07-01, 2023-12-14, 2024-02-18, 2023-12-13] [9999-12-31 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2023-01-15 08:32:59, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00] [2024-06-30 12:01:02, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-11 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2023-12-11 00:00:00, 2024-01-31 00:00:00] [2023-12-10 00:00:00, 2023-12-12 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2015-12-06 00:00:00] [2023-12-20 00:00:00, 9999-12-31 23:59:59, 2026-01-18 00:00:00] [2023-12-18 00:00:00, 2024-01-08 00:00:00, 2023-12-18 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00] [2023-12-12 00:00:00, 2023-12-20 00:00:00, 2024-01-08 00:00:00] [2024-06-30 12:01:02.123000, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2023-12-11 00:00:00, 2025-06-18 00:00:00, 2004-10-15 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00] [2024-01-31 00:00:00, 9999-12-31 00:00:00, 2024-01-08 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00] [2014-03-13 00:00:00, 2024-01-08 00:00:00, 9999-12-31 23:59:59, 2023-12-19 00:00:00, 2023-12-13 00:00:00, 2025-02-17 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00] [2023-12-20 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2023-12-13 00:00:00, 9999-12-31 23:59:59] [2023-12-12 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123123, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 2023-12-18 00:00:00] +264 2026293357 2030570733 449117548 -1237757775 2023-12-16 2026-01-18 2023-12-19 2023-12-10 - u that at think and as what could -- 经销总代理 her because yeah I'll this right we 很细那个 out 0 0 0.000000 2.029400 33.031700 0.000000 2026-01-18 00:00:00 2023-12-14 00:00:00 2009-02-08 00:00:00 2015-07-11 00:00:00 2014-08-12 00:00:00 2023-12-10 00:00:00 2023-12-20 00:00:00 2025-02-17 00:00:00 [] [1, 1, 1] [0, 0, 0, 0, 1, 0, 0, 1, 1, 0] [1, 1, 1, 0, 0, 0, 0, 1, 1, 1] [-126, -105, 1, -128, 0, -128, 87, 127, -90, 0] [79, 1, -128, -36, -128, 1, 1, 0] [-116, -128, -1, 2, 0, -63, 1, -1, 50, -81] [0, 112, -128] [0, -2381, -1, -26738, 3561] [13290, -1, 32767, -32768, -18509] [245, -505, 29753, 1, -24771] [27648, 0, -10958] [-520885663, -1, -419651738, -1909044465, 562500693] [-768711556, -2711870, -898873498, 2147483647, -2147483648, 32679, 1726485329, 2147483647, 1635016385, -1658107629] [] [2147483647, 971053162, -1602272094, -2147483648, -2116629561] [6937473, 0, 0] [1, 4179832, -1] [245, 1, 1816166339, 1074251554, 147483648, 32679, 9223372036854775807, 1908344413, 9223372036854775807, 2077283578] [] [9, -1744275862, 0] [0, -1, -1, -1327290520, -2, 0, -2, -1] [-1, 1154005053, 1] [] \N [-1.200000000000, 1.200000000000, 1.200000000000] [-1.200000000000, 23.173000000000, 1.200000000000] [0.000000000000, 100.020000000000, 0.000000000000, 10.004000000000, 100.020000000000, 73.026300000000, 1.200000000000, 1.200000000000] [63.0591, 100.0200, 37.1782, -1.2000, 300.3430, 40.1230, 55.0454, 300.3430] [1.2000, 0.0000, 0.0000, 1.2000, 1.2000, -1.2000, -1.2000, 1.2000, 100.0200, 100.0200] [40.1230, 1.2000, 23.0478, 1.2000, 1.2000, 300.3430, 300.3430, 77.0297, -1.2000, 100.0200] [40.1230, 93.0537, 25.1446, 6.1507, 300.3430, 26.0852, 81.0795, -1.2000, 0.0000, -1.2000] [-1, 100, 44] [300, 300, 0] [-1, 300, 29, 52, 0, 0, -1, 100, 40, 47] [0, 300, 40, 1, 40] ["更好", "with", "o", "look", "--", "大部分疙瘩新增那你成绩朱晓明李娜住宿费电子为了", "l", "--"] ["o", "your who oh when who", "安徽", "when", "a", "-", "踏踏实实高科汽车做梦没有到货刘金三阳开泰俩年早上贵姓", "know"] ["遮盖跃民庙里珊瑚版算是行踪实话心情汉威", "from he's is then come up had right my", "-", "didn't", "哭泣饱和张峰周哥问过", "?", "--", "back"] ["?", "?", "t", "y", "d", "that", "--", "and for hey the were", "-", "your"] ["go it okay be mean is my didn't time", "佩利鸡巴差距很大量大优惠怎么样喝酒沼气达成率", "所有半天班还是双飞燕关键新密", "out something to who yes we when how don't her", "同方下半年做成一开始诊断淡季", "as could his something want", "don't", "", "can't can of about", "-"] ["don't get is go there like see her", "-", "年底动作这几太难当领导提高网上", "?", "her can't going going but okay were I'm this how"] ["t", "在郑州", "it all you one just or didn't get ok", "-", "", "her what no get did do were come", "", "but", "周岁科技市场", ""] [] ["r", "v", "i", "y", "r"] \N ["u", "n", "o", "v", "k", "w", "q", "d", "b", "n"] ["t", "m", "j"] [2023-12-16, 2024-07-01, 2023-12-17, 9999-12-31, 2026-02-18, 2023-01-15, 2023-12-18, 2024-01-17, 2023-12-17, 2025-02-17] [2023-12-16, 2025-02-17, 2023-12-12, 2024-01-08, 2023-12-14, 2012-08-12, 2023-12-17, 2023-12-15, 2024-01-17, 2023-12-17] [2023-12-19, 2023-12-10, 2014-08-12, 2024-02-18, 2025-02-18, 2023-12-15, 2023-12-13, 2024-01-09, 2014-08-12, 2024-07-01] [2023-12-18, 2024-07-01, 2023-12-12, 2026-02-18, 2027-01-09] \N [2025-06-18 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00] [2024-06-30 12:01:02, 2027-01-09 00:00:00, 2024-01-08 00:00:00] [] [2023-12-16 00:00:00, 2024-07-01 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 9999-12-31 00:00:00] [2024-01-08 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00] [2023-01-15 08:32:59.123000, 2025-02-17 00:00:00, 2023-12-09 00:00:00] [2018-07-20 00:00:00, 2023-12-17 00:00:00, 2011-07-06 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00] \N [2023-12-16 00:00:00, 2024-01-08 00:00:00, 2024-06-30 12:01:02.123000] [2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2023-12-17 00:00:00, 2024-08-03 13:08:30, 2023-12-11 00:00:00] [] +265 200638165 1108704169 -519316339 1702369144 2023-12-14 2023-12-11 2023-12-20 2000-11-18 about 符合烂漫情况沃尔普说好不对外不会差点 ? 内部查杀合作单位铭岳 儿孙们想到 -- - 0 1 40.123000 300.343000 0.000000 40.123000 2023-12-14 00:00:00 2027-01-09 00:00:00 2024-06-30 12:01:02.123000 2026-02-18 00:00:00 2023-12-14 00:00:00 2023-12-18 00:00:00 2023-12-12 00:00:00 2023-12-15 00:00:00 [0, 1, 0, 1, 0] [0, 1, 0, 0, 1] [] [0, 0, 0, 0, 0] [-128, -111, 21, 119, 0, 127, -1, 96, -128, 0] [115, -1, 23] [14, 0, -128, -1, 14, 78, -101, 1] [-58, -128, 124] [32767, 9497, 7851, -8601, 245, -11605, 7193, 13006, 1690, 1] [] [] [-14507, -16059, -32768, 32767, 9236, 14698, -14772, 1, -32768, 32767] [924530644, -2147483648, 2147483647, -1166044203, 2147483647, 2147483647, 0, 449061722, 1808122480, -1] [-1, -399449502, -579365204, -2147483648, 2147483647] [2147483647, 1818267605, 1428209291, 641088625, 1872788837, 147483648, 1907077446, 0, -1, 147483648] [316756898, 32679, 1670056255] [-5600509, 0, -9223372036854775808] [] [9223372036854775807, 5873918271193521319, -1800034856, 0, 1, 32679, 2905966489204205977, -1, 32679, -9223372036854775808] [-9223372036854775808, 215739023368586894, 245, -1, 4117138692663947363] [1719087776, 0, -2, -2, -1, 5, 1067503678, 1, 1, -2] [0, 0, -1193717642] [1, 1, 0, -1, -2, 326922446, 2, -2, 898987665, 122870292] [1, 847648557, -2, 0, -1] [] [1.200000000000, 82.076000000000, 49.009100000000, 1.200000000000, 100.020000000000, 92.154900000000, 300.343000000000, 1.200000000000, -1.200000000000, 64.177900000000] [99.193100000000, 300.343000000000, 300.343000000000, 0.000000000000, 47.120700000000] [300.343000000000, 39.132000000000, 0.000000000000, 14.116700000000, 300.343000000000, 0.000000000000, 300.343000000000, 100.020000000000, 100.020000000000, 39.005000000000] [5.1625, 0.0000, 38.1468, 40.1230, 86.0627] [300.3430, 40.1230, 76.0046, 2.1208, 0.0000, 100.0200, 1.2000, 300.3430, 66.0311, 300.3430] [52.1883, 300.3430, -1.2000, 18.1139, -1.2000, 300.3430, 100.0200, 25.0680, 100.0200, 50.0325] [57.1574, 300.3430, 40.1230, 0.0000, 300.3430] [] [37, 40, -1] [100, 1, 300, 60, 1, 1, 300, 76, 0, 90] [1, -1, 0] ["-", "--", "问题", "?", "n", "I'm or that", "", "自信一览表可是麦迪接口伟业一万优先告诉就爱个", "门窗磁盘柜", "tell go did they when I'm we want tell"] [] ["网友", "i", "亚科键盘鼠标威达创恒款且伴我到供货你家乐凯"] [] ["明华", "-", "众诚问题感悟控制卡不谈金成补上", "联系过", "涨价首创"] ["-", "--", "-", "this say do right mean of hey on", "--"] ["and this that say been all don't", "?", "something not no I'll been do", "who", "you okay some this okay some some can she", "oh not", "be can't", "扩展名对应金牌总代理高杰", "t", "--"] ["--", "安排庄园清理仁信课酬原因的工作", "占有自导", "?", "his", "?", "改掉", "got"] ["v", "h", "t", "v", "w", "b", "i", "j", "v", "n"] ["r", "e", "w"] ["d", "q", "e", "k", "n"] ["v", "o", "d"] [2023-12-14, 2024-01-17, 2024-01-31] [2023-12-11, 2023-12-14, 2025-02-18, 2023-12-13, 2023-12-09, 2024-08-03, 2023-12-11, 2024-02-18] [2024-02-18, 2024-01-19, 2026-01-18] [] [2026-02-18 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59, 2024-02-18 00:00:00, 2023-01-15 08:32:59, 2024-06-30 12:01:02, 2026-01-18 00:00:00] [2024-06-30 12:01:02, 2026-01-18 00:00:00, 9999-12-31 00:00:00, 2023-12-12 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2024-01-09 00:00:00, 2024-07-01 00:00:00, 2023-12-20 00:00:00] [] [] [2025-06-18 00:00:00, 2027-01-09 00:00:00, 2023-12-11 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-09 00:00:00] [2023-12-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [2005-06-10 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00] [2023-12-17 00:00:00, 2023-12-12 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30, 2024-01-19 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00] [2023-12-09 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00] [2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00] [] [] +266 -827218180 0 1715195340 0 2023-12-14 2023-12-09 2024-02-18 2014-08-12 now had she a think come yeah now that - 神仙 - x why here or here 玩玩倒掉不上算是天星太多盘磁带神仙先这样 \N 0 300.343000 -1.200000 0.000000 300.343000 2024-08-03 13:08:30 2023-12-10 00:00:00 2025-06-18 00:00:00 2023-12-13 00:00:00 2027-01-16 00:00:00 2023-12-11 00:00:00 2024-01-19 00:00:00 2023-12-14 00:00:00 [0, 0, 1] [1, 1, 1, 0, 1, 1, 0, 0] [0, 0, 0] [1, 0, 0, 1, 1, 1, 1, 1] [-1, 50, 127] [65, -128, -128] [1, 127, 53, -1, 113, -128, -89, -1, 97, 0] [] [-1, -5192, 32767] [30048, 16062, 11336] [0, 1, -28299, -32768, -32135] [-32768, -32505, -8604] [787213794, 269453166, 0, 32679, 1086778145] [-894322623, 1, -1448569214, -719861388, 1105585997, -2147483648, -1, 1] [-2147483648, -1481702935, -1, 147483648, 32679, -768631195, 1073042519, 1162425516, 2147483647, -2147483648] [-1302104726, -1, 0, 444602763, 147483648, 1970601086, 2147483647, 971613110, 1, -1351611581] [-6353522182140921334, 5880921449835788453, 5252960442806438011] [-1, 32679, 1] [147483648, 9223372036854775807, 5061214833313143790, 1, 32679] [245, 0, 245, 455400973, 1128271, -4489372, -1945123795, 4555473060683533854, 32679, 261683373375747805] [-394199318, -1, 8, -1, 0, -2, 1, 0] [0, 782376788, 513400627, 1, 0, 1, 9, 1, 1, 1] [-2, 1, -1, -1, -972705657, -1, -2, 1] [1, -350623571, -2, -1, 0, 7, -1, 7, -1, 9] \N [300.343000000000, 40.123000000000, 300.343000000000, 40.123000000000, -1.200000000000] [] [-1.200000000000, 36.185600000000, 0.036100000000, 0.000000000000, 100.020000000000, 300.343000000000, 74.021200000000, 25.176100000000, 300.343000000000, 40.123000000000] [] [63.0422, 1.2000, 52.0215, 1.2000, 87.0815] [300.3430, -1.2000, 44.1834, 300.3430, 83.0716, 100.0200, 40.1230, 98.1323] [300.3430, 0.0000, 62.0300, 82.0060, 100.0200] [77, 66, 8, -1, 13, 21, -1, 0, 300, 1] [9, -1, 300, 1, 100, 40, 100, 0] [] [52, 100, 0] ["well so all that would", "", "乔经理"] ["going and will from up all he's see with", "词条", "核对交叉口哦医院网上人情沧桑总经销众佳"] ["办公室标注郑州事先诚信", "把握", "新山邮箱", "r", "也许不说然后母亲面前连续九州开票我还光通", "why", "晓燕", "-"] [] ["v", "here so", "图标", "o", "?"] ["", "going all didn't", "see"] ["time yeah think I'm had no", "c", ""] ["实力我在同学家阶段爱过发送采购下次你怎么去的科汇有限", "e", "国际", "on", "回访农科一次雪城下单甲方琵琶语", "随其自然", "", "--", "客户类型", "入职电厂发财撒很难购物不再性公司最重要几十年鑫辉"] ["c", "z", "j", "p", "a", "z", "a", "l"] ["c", "o", "e", "r", "v", "b", "b", "m", "x", "u"] [] ["m", "o", "p", "d", "r", "r", "t", "t"] [] [2014-08-12, 2023-01-15, 2023-12-19, 2024-06-30, 2025-02-18, 2023-12-13, 2023-12-13, 2024-02-18, 2025-06-18, 2013-07-03] [2023-12-19, 2024-01-08, 2025-06-18, 2024-01-19, 2014-08-12, 2027-01-16, 2023-12-16, 2023-12-19] [2024-01-17, 9999-12-31, 2024-06-30, 2025-06-18, 2024-01-31] \N [2023-12-20 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00] [2024-01-19 00:00:00, 2008-12-26 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00, 2024-01-19 00:00:00] [2023-12-15 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-01-15 08:32:59, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2023-12-12 00:00:00] \N [2024-08-03 13:08:30, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-12-18 00:00:00, 2024-01-31 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59.123000, 2027-01-09 00:00:00] [2023-12-12 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2024-01-31 00:00:00, 2027-01-09 00:00:00] [2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-01-08 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00, 2023-12-10 00:00:00] [2026-01-18 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59] [2027-01-16 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00, 2027-01-16 00:00:00] [2023-12-15 00:00:00, 2025-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-11 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123123] [] +267 1 147483648 2147483647 0 2024-01-17 2026-02-18 2025-02-17 2023-12-09 did 有限广发报表 报表 在那以上卫生考虑英联过节费亚太一鸣赔钱每一小时 got then ? -- 0 1 0.000000 93.122600 1.200000 100.020000 2024-02-18 00:00:00 2023-12-14 00:00:00 2024-07-01 00:00:00 2023-12-16 00:00:00 2027-01-16 00:00:00 2026-01-18 00:00:00 2024-08-03 13:08:30 2023-12-14 00:00:00 [1, 1, 1, 0, 1] [1, 1, 1, 0, 1] [0, 1, 1] [1, 0, 1, 1, 0, 1, 1, 0] [-72, -1, 99] [127, 127, 1] [0, 108, -116, -39, 0, -1, 5, 3, 69, 93] [1, 32, 105, 0, 39] [] [16478, 32767, 32767, 21217, -32768, 8204, 1, 245] [1, 32767, 1] [] [310181289, -1474751800, -1458836259] [1092027761, 2050720583, -1] [-176504329, -1, -2147483648, 1, -2147483648] [32679, -2147483648, 0, -788799691, 1385826470, -2033182653, 915368411, -2147483648] [-3539195, 147483648, 245] [-9223372036854775808, -9223372036854775808, -1409719726773560260] [0, 7193888438975901497, 9223372036854775807] [32679, -7929657647912547035, -9223372036854775808, 0, -1, 1, 9223372036854775807, -9223372036854775808] [-1, 0, 0, 1237019510, -2, 0, -1, 0, 0, -2] [1, -1, 1] [2, 6, 1] [-2, 8, 484537617, -1, -287909092] [] [100.020000000000, -1.200000000000, 1.200000000000, -1.200000000000, 300.343000000000, -1.200000000000, 1.200000000000, 82.150600000000, 0.000000000000, 100.020000000000] [53.164900000000, 1.200000000000, 83.199500000000] [1.200000000000, 12.170300000000, 52.030300000000, 300.343000000000, 0.000000000000] [80.0823, 34.1653, 100.0200, -1.2000, 82.1387, 40.1230, 1.2000, 39.0145, 40.1230, 0.0000] [-1.2000, 1.2000, 1.2000, 0.0000, 100.0200, 44.0390, -1.2000, 1.0438, 20.0348, 1.2000] [77.1209, 100.0200, 88.0141] [4.1896, 76.0105, 92.1417] [67, 40, 0] [-1, 40, 1, -1, 300, 40, 99, 0, 300, 43] [100, -1, 0, 1, 300, -1, 71, 62, 100, 12] [] ["", "-", "去不了", "第三只运输运输鹏飞七匹狼", "?"] ["-", "been something was as a know me really", "五十二万百度大部分没有私人", "I'm", ""] ["mean her you're are who with of how", "-", "get with get you're here her you're can is is", "a", "that's", "什么时间确定新天智能麻烦死首先日信问了", "切换器", "l", "分销潇湘出纳折扣拟在建爱明天找我驱动器我打印机架", "-"] ["--", "", "一米", "-", "l"] ["才好", "", "?", "some", "?"] ["写吧出点特配", "内置", "hey", "-", "or"] [] ["欧阳忽而黑色九九华骏相处打过图像", "because", "帐户"] ["u", "k", "l", "k", "w"] ["g", "t", "k"] [] [] \N [2023-12-20, 2023-12-13, 2023-12-17] [] [2023-12-09, 9999-12-31, 2024-06-30, 2024-01-08, 2023-12-20, 2023-12-11, 2025-02-17, 2023-12-12, 9999-12-31, 2025-06-18] [2023-12-19 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00] [2023-12-18 00:00:00, 2024-01-09 00:00:00, 2015-05-23 00:00:00, 2018-03-09 00:00:00, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59, 2024-01-09 00:00:00, 2023-12-16 00:00:00] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2007-03-03 00:00:00, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2023-12-20 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00] [2024-01-17 00:00:00, 2018-04-26 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2023-01-15 08:32:59, 2014-08-12 00:00:00, 2024-01-31 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2024-01-31 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2023-12-15 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00] [2024-01-31 00:00:00, 2023-12-11 00:00:00, 2024-01-19 00:00:00] [] [2025-06-18 00:00:00, 2024-08-03 13:08:30, 9999-12-31 23:59:59, 2007-06-02 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2026-02-18 00:00:00, 2023-12-13 00:00:00] [] [2027-01-16 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-13 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2024-01-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2025-02-17 00:00:00, 2025-06-18 00:00:00] +268 -942100860 1 -445661833 32679 2023-12-15 2023-12-09 2027-01-09 2025-02-18 柒服装如有实现航务 h ? - - 走过去 p \N 0 6.002600 87.083600 60.036200 0.000000 2023-12-12 00:00:00 2024-02-18 00:00:00 2024-01-17 00:00:00 2025-02-17 00:00:00 2024-08-03 13:08:30 2027-01-09 00:00:00 2026-02-18 00:00:00 2023-12-14 00:00:00 [1, 1, 1, 0, 0, 0, 0, 0] [0, 1, 1, 1, 1, 1, 0, 1] [] [1, 1, 0, 0, 1, 0, 0, 0] [] [-21, -1, -1, 115, -128] [-128, 1, 105, 2, 0] [-28, -128, -95, 111, 7] [-18320, 20067, 1, 9178, 32767] [-3690, 29390, -14584] [-32768, 32767, -29334, -728, 27615, -30924, 32040, 245, 32767, 20925] [-30461, 32767, 1176, -1, -3833, -16136, 245, -15104] [-2147483648, 32679, -1187541181] [32679, 773030488, 147483648, 147483648, 1, 384792039, 1, -1, 147483648, -2147483648] [-1768395835, -228578042, -2147483648, 1207861612, 1057480327, -2147483648, -2147483648, 147483648, -2003897485, 1542623947] [32679, -137475210, 958385294, 312101810, -1903654039, -1, 0, -1411462726] [] [-1, 147483648, 9223372036854775807, -9223372036854775808, -9223372036854775808, -9223372036854775808, 0, 9223372036854775807] [] [0, -556321452, 9223372036854775807] [-2, 8, 6, -2, -1] [-1, -2, -2, 0, 1604349676, 1774029280, -2, 2] [0, 2, 0] [146353093, -2, 6, 1, 1, -1, 79998559, 0] [84.105100000000, 100.020000000000, 11.138900000000, 300.343000000000, 1.200000000000] [0.000000000000, 0.000000000000, 36.192000000000, -1.200000000000, 300.343000000000, 1.200000000000, 40.123000000000, 1.200000000000] [1.200000000000, 40.123000000000, 71.029300000000, 300.343000000000, 0.137800000000, 62.190500000000, 0.000000000000, 1.200000000000, 300.343000000000, 1.200000000000] [300.343000000000, 100.020000000000, -1.200000000000, 100.020000000000, -1.200000000000] [97.1442, 38.1538, -1.2000, -1.2000, 67.1920, -1.2000, 5.0565, 28.0346] [] [300.3430, 1.2000, 300.3430, 8.1722, 300.3430] [] [1, 40, 46, 300, 41] [0, 300, 16] [0, 40, 88, 33, 100] [1, 1, 300, 1, 0, -1, 100, 0, 1, 100] [] ["", "余下只能代表孔艳淀粉回报联系人最美丽照顾等等河南省郑州市", "-", "投资", "--"] ["what", "玛法", "中原北", "疑问辽源名称灵生很难说", "", "马后", "-", "--"] [] ["?", "?", "are do her with we get are that she"] ["really", "-", "on who", "--", "why"] [] ["and from of to not", "right", "or", "what oh here", "and they could have know is some"] ["w", "w", "l"] \N [] ["p", "w", "h", "j", "c"] [2027-01-09, 2025-02-18, 2023-12-09, 2023-12-16, 2023-12-19, 2023-12-16, 2023-12-16, 2023-12-09] [2023-01-15, 9999-12-31, 2023-12-10, 2023-12-18, 2023-12-16, 2023-01-15, 2024-07-01, 2023-12-16] [2024-01-17, 2023-12-14, 2023-12-13] [2023-12-15, 2017-07-18, 2024-01-31] [2024-01-17 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2023-12-09 00:00:00, 2025-06-18 00:00:00] [2024-01-31 00:00:00, 9999-12-31 23:59:59, 2024-01-19 00:00:00] [] [2023-12-15 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02, 9999-12-31 00:00:00, 2000-05-28 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00] [2023-12-09 00:00:00, 2023-12-19 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2024-06-30 12:01:02.123000, 2024-01-09 00:00:00, 9999-12-31 23:59:59, 2023-12-16 00:00:00] [] [2023-12-20 00:00:00, 2023-12-20 00:00:00, 2023-12-14 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-11 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00] [2023-12-11 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2010-07-16 00:00:00, 2023-12-16 00:00:00] [2024-02-18 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2027-01-09 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2027-01-16 00:00:00] [] [2025-06-18 00:00:00, 2023-12-10 00:00:00, 9999-12-31 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00] +269 -2147483648 0 0 0 2025-02-18 2023-12-19 2024-06-30 2023-12-15 just you're know were here now will he's 姚明 追到广发线缆南三环彩页勾丽娜 已给 say how what can't -- 岂能 \N 1 -1.200000 81.034700 1.200000 -1.200000 2026-01-18 00:00:00 2027-01-16 00:00:00 2024-02-18 00:00:00 2024-01-08 00:00:00 2024-08-03 13:08:30 2024-08-03 13:08:30 2027-01-09 00:00:00 2023-12-10 00:00:00 [1, 1, 1, 0, 0] \N [1, 1, 1, 1, 1] [1, 1, 1] [1, 3, -33] [0, 127, -128, -113, 27, -124, -128, 1, -1, 4] [-54, 1, 0, -112, 127] [109, -43, 1, 0, -124, -69, 9, 1, 8, -128] [32767, 245, -32768, 1, 245, -1, 245, -27040] [8946, 245, 1, -26711, 0, -1, 245, -32768, 1, 27509] [] [29902, 29044, 32152, 14309, 7213, -32768, -1, 32767, 1, 12778] [-92962590, -1, -1, 147483648, 32679, 1, 680114895, 41676391, -249173035, -1] [1, 1, -1491274395, 147483648, -1, 1786496152, -1, -141900829] [195067664, -1524946907, 147483648, 1094544078, 147483648, 32679, 1, 147483648, -751301488, 32679] [] [50155956825696787, 757770856933941066, -1, 1611953894807250218, 4099638, -1877987793382958528, 147483648, 147483648] [147483648, 245, 32679, -5790528, -8312715130356887145, 32679, 1, 0] [-1, 6086250, 147483648, 9223372036854775807, 245] [-1, -9223372036854775808, -1834223356, -844413998, 147483648] [-1180993001, 0, 0, -2, 0] [-1, 1, 1, -1923367341, 0, 1, -1, 1] [439155884, -848501023, 1437764641, 2, -1, 1, -2, 0] [] [1.200000000000, 22.195400000000, 1.200000000000] [22.145400000000, 100.020000000000, 300.343000000000, 100.020000000000, 1.200000000000, 60.056000000000, 20.026500000000, -1.200000000000, 40.123000000000, 1.200000000000] [83.027900000000, 300.343000000000, -1.200000000000, 93.075900000000, 300.343000000000, 8.080700000000, 300.343000000000, -1.200000000000] [100.020000000000, 52.121000000000, 300.343000000000, 300.343000000000, 1.200000000000, 300.343000000000, 300.343000000000, 100.020000000000, -1.200000000000, 79.006800000000] [-1.2000, 0.0000, 300.3430, 0.0000, 1.2000, -1.2000, 0.0000, 300.3430] [7.1206, 52.1517, 95.0875, 7.1308, 42.0179, 300.3430, 40.1230, -1.2000] [-1.2000, 26.0817, 100.0200, 73.0209, 45.1977] [] [96, 300, 40] [-1, -1, -1, 100, 0, 40, 1, 0, 86, 1] [0, -1, 0, 68, 100, 100, 100, -1, 74, 40] [-1, 54, 100] ["?", "?", "--"] ["me", "?", "交货", "名单压在生后废话", "in"] [] ["?", "", "that"] \N ["?", "-", "去爱", "hey", "-", "v", "?", "见你"] ["-", "-", "--", "x", "good tell mean can't will how for"] ["那行聂柯县城分别还得国标有待于运行攻防", "那次", "飞机"] [] ["x", "o", "p", "z", "u", "q", "w", "j"] ["p", "c", "l"] [] \N [] [9999-12-31, 2027-01-09, 2025-06-18, 2023-12-15, 2023-12-18, 2026-01-18, 2023-01-15, 2024-01-08, 2026-02-18, 2026-01-18] [2023-12-17, 2024-06-30, 2024-01-08, 2023-12-10, 2023-12-17, 2025-02-18, 2024-01-19, 2025-02-17, 2027-01-16, 2023-12-14] [] \N [2023-12-11 00:00:00, 2026-02-18 00:00:00, 2025-02-18 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00] [2024-07-01 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [2026-01-18 00:00:00, 2023-12-09 00:00:00, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2026-02-18 00:00:00, 9999-12-31 00:00:00, 2024-01-09 00:00:00] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 9999-12-31 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00, 2023-12-14 00:00:00, 2010-12-24 00:00:00, 2024-01-17 00:00:00, 2011-04-05 00:00:00] [2024-01-08 00:00:00, 2023-12-14 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-18 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2024-01-09 00:00:00] [2014-08-12 00:00:00, 2023-12-12 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2026-01-18 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00] [] [2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2023-12-15 00:00:00] [2023-12-15 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2024-01-08 00:00:00] +270 -1565635653 147483648 -784352779 -1 2023-01-15 9999-12-31 2023-01-15 2027-01-16 is that then go what have yes go can't it's as can here k 有限公司汉化直销郑州折叠系统几台短语控制卡 也是 条件 信海 1 0 0.000000 0.000000 0.000000 100.051200 2023-12-14 00:00:00 2024-01-19 00:00:00 2024-02-18 00:00:00 2023-12-09 00:00:00 2027-01-09 00:00:00 2023-12-19 00:00:00 2024-02-18 00:00:00 2025-02-17 00:00:00 [] [] [1, 1, 0, 0, 0, 0, 0, 1, 1, 1] [0, 1, 1, 0, 0] [7, 127, 1] [-1, 1, 0, -128, -128, 127, -1, 48, 127, -1] [16, 114, 127, 1, -127, 6, 19, 21, -128, -28] [0, -1, 0] \N [245, -32768, 1, 19390, 1] [-13547, -1, 245, -8443, 32767, 1, 25278, 1, -32768, -12425] [9071, -1, 26912] [32679, 176488874, 1, 1244816482, 1445224322] [147483648, 2147483647, 2147483647, -93095277, -2147483648, 1, 461977622, 2147483647] [-2147483648, 147483648, 147483648] [] \N [1764804721, -1847228767, 3477899596856565616, -1, -9223372036854775808, -9223372036854775808, -5495351607995757934, -1515147708043304432] [147483648, 1, 9223372036854775807, 363583, 9223372036854775807] [] [0, -1130281530, 1] \N [-2, 0, -2, 3, 28871955, 1060040114, 1, 1, 0, 7] [-107977625, 0, 0, -2, 0] [48.095600000000, 40.123000000000, 40.123000000000] [-1.200000000000, 40.123000000000, 21.192500000000, 0.000000000000, 40.123000000000, 0.000000000000, 49.059900000000, 40.079300000000, 1.200000000000, 0.000000000000] [] [] [100.0200, 40.1230, 0.0000, 1.2000, 300.3430, -1.2000, 26.0677, 300.3430] [0.0000, 0.0000, 0.0000, 4.0995, 85.0222, 40.1230, 300.3430, 300.3430] [40.1672, 100.0200, 0.0000, 1.2000, -1.2000] [] [] [] [81, 40, 40, 96, 40] [0, 0, 100, 0, 49] ["--", "here", "金鳞建议网景认可其他分数韩鸽飞玩游戏是你或房地产", "would that think going her some", "所作"] ["三石", "", "now", "全勤奖刷卡顺河路老刘同力", "线程呢个初次", "-", "玲玲快点每个人很多那样子预算元整好心", "some"] [] ["because", "--", "", "are it's what know you", "忘不了"] ["", "to", "were at if back or his at", "核心", "is some think mean not of I'll yes from", "because well right hey out from well mean know he", "西开发事情刷卡青楼带来景峰协作", "最深刻对自己礼貌请你吃饭子夜洛宁他人", "x", "-"] ["can because yeah say or", "创世纪蓝牙东大街建议不再随北风欢迎欢迎佳成", "-", "then here going one he going think and because", "?"] ["no this", "-", "get", "have yeah how that's had there are will want", "--", "it's go did something", "不用之间", "鼠标影子怎么着新乡市高科", "价格上耐心而蓝牙利润", "倪玲梅"] [] ["v", "w", "z"] ["x", "s", "n", "b", "s", "f", "q", "p", "r", "h"] ["j", "t", "n", "i", "m", "x", "f", "a", "q", "p"] ["j", "v", "y"] [] [2024-01-19, 2023-12-16, 2024-01-08, 2023-12-18, 9999-12-31] [] [2023-12-19, 9999-12-31, 2024-01-08] [2003-05-25 00:00:00, 2008-04-10 00:00:00, 2017-06-24 00:00:00, 2023-12-20 00:00:00, 2023-12-14 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2024-06-30 12:01:02] [2024-08-03 13:08:30, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00] [2026-01-18 00:00:00, 2014-08-12 00:00:00, 2023-12-20 00:00:00] [2026-01-18 00:00:00, 2023-12-11 00:00:00, 2023-12-10 00:00:00] [2023-12-19 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2024-07-01 00:00:00] \N [2025-06-18 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-10 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-14 00:00:00] [2024-01-08 00:00:00, 2025-02-17 00:00:00, 2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000] [2017-02-27 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59, 2024-01-17 00:00:00, 2014-08-12 00:00:00] [2024-01-31 00:00:00, 2016-05-02 00:00:00, 2023-12-10 00:00:00, 2023-12-20 00:00:00, 2023-12-16 00:00:00, 9999-12-31 23:59:59, 2023-01-15 08:32:59.123123, 2023-12-15 00:00:00] [] [2027-01-09 00:00:00, 9999-12-31 00:00:00, 2023-12-11 00:00:00, 2027-01-16 00:00:00, 2024-01-08 00:00:00, 2023-01-15 08:32:59.123123, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00] +271 \N -143401454 1971545478 899631563 2023-12-16 2025-06-18 2026-02-18 2024-01-08 ? ? this have know the why time could - ? 招待费单子瑞信线上刻骨铭心完美盛源对方 we why no your -- 0 0 87.121800 0.000000 20.101900 28.117700 2024-08-03 13:08:30 2024-02-18 00:00:00 2023-12-15 00:00:00 2025-02-17 00:00:00 2024-06-30 12:01:02.123000 2024-06-30 12:01:02.123000 2024-06-30 12:01:02.123000 2023-12-09 00:00:00 [] [] [1, 1, 1, 0, 1, 0, 1, 1] [0, 0, 1, 1, 1, 0, 1, 1, 1, 1] \N [] [7, 97, 1, 5, -128, -128, 127, 0] [-128, 117, 2] [32767, 1, 1, -15662, -9843] \N [-19362, 32767, 0, -32768, 1] [-1, 1, 20913] [32679, -1, -1, -1, 2147483647, 147483648, 1340663753, -1] [918468032, 2096861511, 1] [-1, 446281776, 323068165] [-1943547035, 775567917, 2147483647, -1, 32679] [49020245, -1, 245, -6156713, -5058464234380662116] \N [279470, 245, 147483648, 2011950635, -1] [-4317073, -4631623178789200856, 0, 245, 32679, 9223372036854775807, -1326842272, -2627835096741768803, 245, -5683535725693372255] [-2, 1, -1, 6, -1] [5, 6, -2] [1345681569, -1492867217, 4, -1, -451622537, -413668938, 0, -1, 4, -1] [1, -1432584601, -2, 1, 1, -1056136676, 649209431, 2054396492] [68.096200000000, 50.140000000000, 50.135900000000, 40.123000000000, 0.000000000000, -1.200000000000, 66.034300000000, 300.343000000000] [300.343000000000, -1.200000000000, 1.200000000000, 40.123000000000, -1.200000000000, -1.200000000000, 73.023000000000, 42.053900000000] [40.123000000000, 100.020000000000, 59.001900000000] [40.123000000000, 62.131000000000, 0.000000000000, 51.156200000000, 40.123000000000, 100.020000000000, -1.200000000000, 100.020000000000] [0.0000, 92.1275, 100.0200, 100.0200, 300.3430, 38.1107, -1.2000, -1.2000] [0.0000, 59.1977, 0.0000] [11.0646, 41.1668, 3.1265, 1.1716, 83.1446] [300.3430, 0.0000, 59.0496, 300.3430, 1.2000, 1.2000, 18.1647, 40.1230, 100.0200, 100.0200] [0, 72, 100, 8, 40, 100, 1, 1, 98, 40] [] [25, 100, -1, 93, 1, 1, 0, -1, 25, 300] [76, 300, 81] [] ["we get", "didn't did would yeah at me", "can to can", "", "有时间", "?", "-", "can"] ["-", "that's", "申腾拜托好卖出点客户化验有种我在同学家易插拔关机", "that", ""] [] ["经济", "well", "表情", "?", "许愿树带你军歌感受无关", "one", "引进过来天慧下游", "did I'll he she up been for", "is right just and this or do have is", "单子"] ["?", "?", "-"] ["?", "it's", "him", "-", "is a tell go he okay will he", "you're so think just if so you're oh", "小玲本来差价", "--", "难受", "差价"] [] ["q", "k", "u", "f", "g"] ["s", "k", "q", "q", "n"] ["m", "r", "j", "n", "v"] [] [2025-02-18, 9999-12-31, 2024-01-19] [2027-01-09, 2024-06-30, 2023-12-13, 2024-08-03, 2024-02-18, 2026-01-18, 2023-12-18, 2023-12-16, 2026-01-18, 9999-12-31] [2025-06-18, 2025-06-18, 2027-01-16, 2023-12-15, 2024-02-18, 2023-12-12, 2025-06-18, 2023-12-11, 2011-07-26, 2025-06-18] [2023-12-16, 2023-12-11, 2023-01-15, 2014-08-12, 2023-12-14, 2024-01-17, 2023-12-10, 2023-12-19] [2023-12-20 00:00:00, 2023-12-10 00:00:00, 2023-12-09 00:00:00, 2023-12-14 00:00:00, 2024-01-19 00:00:00] [2023-12-11 00:00:00, 2025-06-18 00:00:00, 2008-05-13 00:00:00] [9999-12-31 23:59:59, 2023-12-10 00:00:00, 2023-12-12 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2026-02-18 00:00:00, 2023-12-09 00:00:00, 2023-01-15 08:32:59, 2025-02-18 00:00:00, 2024-06-30 12:01:02] [2023-12-18 00:00:00, 2023-12-14 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59] [2023-12-15 00:00:00, 2024-06-30 12:01:02.123000, 2023-01-15 08:32:59.123000, 2024-01-17 00:00:00, 9999-12-31 23:59:59] [2023-12-16 00:00:00, 2025-02-18 00:00:00, 2023-12-14 00:00:00] [2023-12-09 00:00:00, 2024-01-19 00:00:00, 2024-01-31 00:00:00, 2014-08-12 00:00:00, 2023-12-10 00:00:00, 2001-05-20 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00] [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00] [2026-02-18 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2026-02-18 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00, 2026-01-18 00:00:00, 2023-12-19 00:00:00, 2024-01-19 00:00:00, 2023-12-16 00:00:00] [2025-02-17 00:00:00, 2023-12-18 00:00:00, 2014-08-12 00:00:00, 2023-12-18 00:00:00, 2026-01-18 00:00:00, 2014-08-12 00:00:00, 2023-12-17 00:00:00, 2000-09-16 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [2027-01-09 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-12-16 00:00:00, 2023-12-12 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-08 00:00:00] [2025-06-18 00:00:00, 2023-01-15 08:32:59.123123, 2026-01-18 00:00:00, 9999-12-31 23:59:59, 2023-12-14 00:00:00] +272 -861637827 32679 -2147483648 1 2024-01-17 2024-02-18 2026-02-18 2014-08-12 询问漫长重点偷出肛门写在最上面第二次 代玉 why 舞阳好人坏人招生衣服充满不用聊聊很早理想关于 能不能医药回家 it's -- 地方那个 0 0 0.000000 -1.200000 40.123000 300.343000 2027-01-09 00:00:00 2013-05-06 00:00:00 2027-01-16 00:00:00 2023-12-15 00:00:00 2023-12-14 00:00:00 2023-12-12 00:00:00 9999-12-31 23:59:59 2026-01-18 00:00:00 [0, 0, 1, 1, 1, 1, 1, 0, 1, 1] [1, 0, 1, 1, 1, 1, 1, 0] [] [0, 1, 1, 1, 0] [-1, 1, 23] [] [0, 3, -1, -100, 127] [] [32767, 7644, -16965, 0, 32767] [1, -5272, 9166] [] [-32768, -27317, 21708, 19134, 29008, 4478, 1, 0] [-82553313, 2147483647, 2147483647, 2147483647, 1797605742] [-1069583647, -1756890040, 0, 266419903, -2147483648, 1, -2147483648, 0, 1164055644, -2147483648] [-671285810, 147483648, 952081014, 32679, 1320124986] [-2147483648, 2147483647, -933154581] [] [-9223372036854775808, 6200071811880679307, 1, 0, 863722716] [5948714, 32679, 32679, 9223372036854775807, 147483648] [-3974174589477889886, 147483648, -6596395167018087002, -5004358657269923232, 1, -1697194380670801901, -1, 1, 0, 1] [] [0, 383490456, -1, 0, -1, 1, 0, -2, -2, 2122726499] [] [-1, -2, 0] [37.069200000000, 36.192300000000, 1.200000000000, 100.020000000000, -1.200000000000, 72.104800000000, 1.200000000000, 0.000000000000, 70.175600000000, 1.200000000000] [97.062400000000, -1.200000000000, -1.200000000000, 35.118000000000, 100.020000000000, 100.040800000000, 40.123000000000, 39.172700000000, 0.000000000000, 80.014300000000] [91.085700000000, 38.121000000000, 100.020000000000] [39.141200000000, 72.086200000000, -1.200000000000, 14.037300000000, 96.162100000000, 1.200000000000, 23.072900000000, 71.148400000000, 43.051300000000, 1.200000000000] [63.1568, 1.2000, 14.1207, 1.2000, 93.1252] [100.0200, 86.1991, 40.1230] [99.0904, 59.0435, 40.1230, 17.0908, 100.0200, 1.2000, 34.0239, -1.2000, 300.3430, -1.2000] [40.1230, -1.2000, -1.2000, 16.0128, 100.0200] [0, 100, 300, 0, 100] [22, 300, -1, 0, 58] [] [0, 40, 31, 1, 57] [] \N ["okay", "one", "怎么着新买程序取悦惭愧输得起", "m", "走之前生物石龙活动"] ["郭晓卖卖机型", "星星中档核好说话王峰所要求温总还在博源", "e"] ["-", "--", "南阳倪玲梅之前参数表进账带库周经理", "f", "-"] ["字库文件", "?", "ok here are are don't", "壹佰", "通过"] ["were how why this get could were", "his is who why", "could", "成绩", "?", "", "some", "?", "oh", "c"] ["套件", "", "--", "?", "w", "tell when", "", "-", "back oh come", "--"] ["c", "i", "y", "p", "v", "t", "u", "h"] \N ["h", "s", "o", "r", "f", "p", "t", "i", "s", "i"] [] [2023-12-10, 2024-01-08, 2005-08-22, 2024-02-18, 2025-06-18] [2025-06-18, 2024-02-18, 2023-12-18] [2025-02-18, 2014-08-12, 2027-01-09, 2023-12-15, 2023-12-17] [] [2023-12-20 00:00:00, 2024-01-19 00:00:00, 9999-12-31 23:59:59] [] [2024-02-18 00:00:00, 2026-01-18 00:00:00, 2023-12-15 00:00:00, 2024-01-17 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2002-07-17 00:00:00, 2024-01-19 00:00:00] [2025-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00] [2025-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2025-06-18 00:00:00, 2024-01-08 00:00:00, 2024-01-08 00:00:00, 2023-12-14 00:00:00, 2027-01-16 00:00:00, 2025-06-18 00:00:00] [2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00, 2023-12-15 00:00:00, 2023-12-16 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00] [2024-01-17 00:00:00, 2023-12-09 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2027-01-09 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2023-12-16 00:00:00] [2014-08-12 00:00:00, 2023-01-15 08:32:59.123000, 2023-01-15 08:32:59.123000] [2019-06-15 00:00:00, 2024-01-31 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00] [2024-08-03 13:08:30, 2023-12-18 00:00:00, 2014-08-12 00:00:00, 2024-08-03 13:08:30, 2027-01-16 00:00:00] [] [2027-01-16 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00] +276 2147483647 147483648 -838648730 -1129987166 2023-12-16 2023-12-13 2025-06-18 2024-01-09 ? 沈总 f 其实 I'm ? 所有出不了启恒说明张海燕待定空间 根本 0 1 100.020000 0.000000 100.020000 0.000000 2024-01-08 00:00:00 2026-02-18 00:00:00 2023-01-15 08:32:59.123000 2027-01-09 00:00:00 2014-08-12 00:00:00 2027-01-16 00:00:00 2023-12-12 00:00:00 2026-02-18 00:00:00 [0, 1, 1, 1, 1, 0, 1, 1, 1, 1] [1, 1, 1, 1, 1, 1, 1, 1, 0, 0] [1, 1, 0, 0, 0, 1, 1, 1] [1, 1, 1, 0, 1] [-128, 0, 1, -128, -13, 3, 45, 0, -128, 1] [127, 8, -77, -94, 1] [] [-1, -14, -1] [26168, -10642, -28908, 245, 1021, 19769, -32768, 1834] [-24462, 7730, 17656, 6551, -23693, -18399, 31827, 3639, -1, 32767] [-32768, -21976, -1984, -1, 245] [-8898, -17658, 25753, 0, 245, -3059, 5807, -12258, 25065, 245] [0, 0, -644562874, 32679, 0, 110542730, 32679, -2147483648] [] [1, -2147483648, 32679] [2037666783, -523580216, -2147483648, 147483648, -312408527] [-9223372036854775808, -1235966203, 32679, 147483648, 147483648, 0, 32679, 147483648] [245, -5674277979277106325, 147483648] [] [9223372036854775807, 9223372036854775807, -362362618812897469] [] \N [0, -529929133, -2] [] [47.130100000000, 300.343000000000, 40.123000000000, 40.123000000000, 43.105800000000] [] [] [100.020000000000, 300.343000000000, 42.051700000000, 40.123000000000, 71.181900000000, 100.020000000000, 300.343000000000, 300.343000000000, 1.200000000000, 37.192100000000] [33.0195, 300.3430, 100.0357, 16.1194, 0.0000] [300.3430, 90.0509, 100.0200, 100.0200, 22.0085, 300.3430, -1.2000, -1.2000] [58.0760, 300.3430, 40.1230, -1.2000, 1.2000, 40.1230, -1.2000, 4.1967, -1.2000, 40.1230] [-1.2000, -1.2000, 100.0200, 0.0000, -1.2000, 89.1647, 100.0200, -1.2000] [40, 0, 300] [77, 52, 0, 31, 40] [] [58, 1, 1, 0, 100] ["she like", "-", "it", "-", "周岁"] ["", "速达", "hey of"] ["市委", "-", "", "正版", "?", "绿城", "she who", "-"] [] [] ["be", "about ok out one know from okay from go time", "?"] ["忽而", "are if", "", "-", "?"] ["华栋见到李东无所谓无限总经销", "", "it", "-", "--"] ["n", "a", "v", "g", "r", "t", "n", "o"] ["q", "t", "t"] [] [] [2023-12-16, 2026-01-18, 2027-01-09, 2025-02-18, 2023-12-15] \N [2024-06-30, 2024-08-03, 2024-07-01, 2023-12-19, 2023-12-13] [2025-02-17, 2023-12-13, 2024-01-09] [2023-12-15 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59, 2027-01-16 00:00:00, 2023-12-15 00:00:00] [2023-12-13 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00, 2023-12-19 00:00:00, 2025-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-09 00:00:00] [2003-06-25 00:00:00, 2016-10-10 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00] [2027-01-16 00:00:00, 2023-12-09 00:00:00, 2023-12-11 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00] \N [2024-01-17 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00] [2024-01-08 00:00:00, 2017-02-08 00:00:00, 2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-13 00:00:00] [2023-12-09 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00] [2023-12-14 00:00:00, 2023-12-13 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00] [2026-02-18 00:00:00, 2024-08-03 13:08:30, 2026-01-18 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2023-12-11 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2023-12-15 00:00:00] [2023-12-14 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00, 2023-12-12 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00, 9999-12-31 23:59:59] +277 -1 1829776471 -1 1653474079 2023-12-16 2023-12-18 2023-12-15 2024-01-09 really now that's were back was when what 佩服我们政务 小故事 get think would just something 三阳开泰 1 0 1.200000 -1.200000 300.343000 10.118600 2023-12-19 00:00:00 2024-08-03 13:08:30 2027-01-09 00:00:00 2023-12-19 00:00:00 2024-08-03 13:08:30 2023-12-17 00:00:00 2023-12-16 00:00:00 2023-12-12 00:00:00 [0, 0, 0, 1, 0] [1, 1, 0, 1, 1] [1, 1, 0, 1, 0] [] [] [37, 5, 0, 0, 70, 5, 5, 86, -42, 1] [] [127, -109, -75] [0, -8443, 245, -1, 14335, 245, -12341, 7631, 32767, 28264] [-25713, -8925, 19767, -32768, 0, 0, 32216, 0] [-32768, 32767, -4511] [32767, 1, -19302] [1144381807, -2147483648, 14291333] [-1244685146, 0, -2147483648] [147483648, 1478994999, 32679, 32679, 147483648, -771813212, 1158928631, 32679] [147483648, 0, 1565072238] [-1, 245, 245, 0, -5456278, 0, 9223372036854775807, 2659287593464110071, 9223372036854775807, -9223372036854775808] [] [1, 521040860, 0] [-2085317555009290567, 610371512, -3637055] [] [1, -1, 0, 0, 0, -1761101134, 1467527214, -2, -1294704980, 1] [870474423, 0, -1796270919] [0, -1996077275, 2, -2, -1] [100.020000000000, 12.138700000000, 100.020000000000, 100.020000000000, 94.012000000000, 64.073900000000, 32.100000000000, 29.049900000000, 0.000000000000, 94.041100000000] [] [86.152600000000, -1.200000000000, 40.123000000000, 64.037600000000, 100.020000000000, 15.167600000000, 0.000000000000, 40.118300000000] [9.047600000000, 100.020000000000, -1.200000000000, 100.020000000000, 300.343000000000, -1.200000000000, 300.343000000000, 16.192500000000, 16.119600000000, 300.343000000000] [1.2000, 77.0104, 97.0831, 300.3430, 40.1230, 0.0000, 40.1230, 63.1638, 28.0229, 35.0504] [48.0707, 99.0034, 43.1724, 22.0658, -1.2000, 83.0525, 0.0000, 95.0343, 44.1270, 0.0000] [100.0200, 41.1225, 0.0000, 25.1330, 80.1878, 58.0312, 100.0200, 48.1466, -1.2000, 100.0200] [-1.2000, 0.0000, 1.2000, 1.2000, 40.1230] [40, 54, 43, 1, 1, 100, 40, 40] [300, 1, 47] [] [15, 36, 300, 36, 98, 100, 77, 300] ["--", "今天", "-"] \N ["压力", "三级旭日普通不咱优势信用社平常允许有空吗李总", "you your"] ["up", "杂请你们优惠一样", "-", "were is he know", ""] ["", "didn't my hey with up get", "you", "?", "", "j", "if", "could I'll well why me is I'll as like"] \N ["", "作假明辉艾蕊前面骗人人民蠢材自导", "you're was well hey some I'm been", "单子下市场虚妄留不住", "here do you're", "a you're that we out but all", "go", "he"] ["中文", "?", "one yes you one", "-", "", "-", "水利难过跃民卖友纸业贵姓排列怎么样", "不错"] ["f", "q", "g", "j", "z", "x", "w", "y", "u", "o"] ["n", "v", "r", "v", "t", "d", "q", "r"] [] ["t", "v", "o"] [] [2025-02-18, 2023-12-20, 2023-12-17, 2024-01-08, 2023-12-13, 2024-01-08, 2026-01-18, 2024-02-18, 2024-07-01, 2024-02-18] [2023-12-09, 2025-06-18, 2024-01-17, 2027-01-16, 2023-12-19, 2025-06-18, 2023-01-15, 2023-12-17, 2023-12-13, 2025-06-18] [2024-02-18, 2025-06-18, 2027-01-16, 2023-12-11, 2014-08-12, 2023-12-20, 2024-02-18, 2024-01-31] \N [] [2024-01-31 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00] [2024-01-31 00:00:00, 2027-01-09 00:00:00, 2025-02-18 00:00:00, 2023-12-17 00:00:00, 9999-12-31 00:00:00] [2024-02-18 00:00:00, 2023-12-16 00:00:00, 2024-01-09 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2023-12-13 00:00:00] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2027-01-09 00:00:00, 2024-07-01 00:00:00, 2024-01-31 00:00:00, 2023-12-18 00:00:00, 2024-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2024-01-31 00:00:00] [2024-01-08 00:00:00, 2023-12-15 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00, 2023-12-15 00:00:00, 2025-02-17 00:00:00, 2025-02-17 00:00:00, 2024-08-03 13:08:30, 2023-01-15 08:32:59.123000, 2023-12-10 00:00:00] [9999-12-31 23:59:59, 9999-12-31 00:00:00, 2023-01-15 08:32:59.123000] [2024-01-31 00:00:00, 2023-12-14 00:00:00, 2012-04-15 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00] [] [2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-12-20 00:00:00, 2025-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00] +278 -1 147483648 -1895613943 1 2014-08-12 2023-12-10 2024-01-09 2024-02-18 ? for would - ? he think will were b 发吧 \N 0 300.343000 46.018300 100.020000 100.020000 2023-12-12 00:00:00 2023-01-15 08:32:59.123000 2027-01-16 00:00:00 2024-01-31 00:00:00 9999-12-31 00:00:00 2023-12-09 00:00:00 2024-07-01 00:00:00 2025-02-18 00:00:00 [0, 1, 1] \N [1, 0, 0] [1, 1, 1, 1, 1, 0, 0, 0] [1, 7, 7, 2, -128, 1, -1, 127] [0, 3, -128, 89, 0, 127, 0, 127, 0, -80] [40, -77, 127, 5, 0, 127, 5, -104] [4, 98, 0] [3439, -32753, -6170, 0, 23298, -22276, -5889, 15469] [0, 245, 32767, 18162, 1, 1, -1, -25931, -21044, 8450] [-32768, 13912, 32767] [] \N [] [-542106640, 32679, 1844773729] [] [9223372036854775807, 245, -1] [1792979631, -2628200041205504199, -9223372036854775808, 1, 2606571911828787788] [-1341191711, 147483648, -9223372036854775808] [9223372036854775807, 245, 1, -1, 1, -619249427, -6932235, 9223372036854775807, 32679, 1825432870325793363] \N [-1, -1470746829, -2095528210, -1, -1] [] [-1544761727, 0, 1, 0, 4, 1, -1458703097, -1472056035] [0.000000000000, 100.020000000000, 300.343000000000, 30.161900000000, -1.200000000000, 100.020000000000, 0.000000000000, 1.200000000000, 100.020000000000, 0.000000000000] [37.037200000000, 74.079100000000, 8.008200000000, 300.343000000000, 100.020000000000] [-1.200000000000, 95.090400000000, 68.139600000000] [100.020000000000, 0.000000000000, 0.000000000000, 300.343000000000, 28.138900000000, 1.200000000000, 300.343000000000, 1.200000000000, 40.123000000000, 72.058500000000] [40.1230, 46.1221, 1.2000, 300.3430, 1.2000] [40.1230, -1.2000, 1.2000, 72.0227, 300.3430] [86.1544, 7.1245, 90.0854, 300.3430, -1.2000, 55.1938, 83.0769, 0.0000, 1.2000, 300.3430] [] [54, -1, 300, 1, 300, 30, 100, -1, 100, -1] [1, 0, 86, 100, 300] [0, 300, 19, 40, 82] [27, 100, 6, 24, -1, 12, 100, -1] ["no there yes we", "a", "供货洗衣服", "have who oh going the", "yes like hey if her about of is"] ["?", "都还", "-", "all just okay so him", "大公司", "because", "?", "here of we yeah", "-", "--"] ["吉林", "孙海洋", "no", "烟火一下阿前期承认了户我们的考试绝对", "", "?", "", "-", "广告词", "one"] ["k", "because", "n", "one then you're in the come", "something good in now my", "黑帮带来气人百度处于赔钱只要", "吃住", "含有瀚海办完升职", "", "代理价格看法利万姑娘"] [] ["from", "方案麻烦不起东风路郑州市电器恶评", "-"] ["out", "-", "w", "-", "国标", "that", "综合", "洗液", "y", "下次"] [] ["g", "w", "q"] ["u", "a", "w", "y", "k"] [] [] [2025-06-18, 2023-12-10, 2024-06-30] [2023-12-10, 2023-12-19, 2023-12-14, 2024-01-19, 2023-12-10, 2024-02-18, 2023-12-17, 2024-01-08, 2025-02-18, 2024-01-19] [9999-12-31, 9999-12-31, 2023-12-20, 2025-06-18, 2024-01-09] [2025-06-18, 2023-12-19, 2023-12-16] [2025-06-18 00:00:00, 2014-08-12 00:00:00, 2024-07-01 00:00:00] [2023-12-13 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-11 00:00:00] [2023-12-12 00:00:00, 2024-06-30 12:01:02, 2024-08-03 13:08:30, 2023-12-11 00:00:00, 2024-01-08 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2023-12-16 00:00:00] [2023-12-15 00:00:00, 2025-02-17 00:00:00, 2023-12-13 00:00:00, 2023-12-15 00:00:00, 2023-01-15 08:32:59] [2026-02-18 00:00:00, 2024-01-17 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2025-02-17 00:00:00, 2023-12-16 00:00:00, 2023-12-15 00:00:00, 2024-07-01 00:00:00] [2014-08-12 00:00:00, 2023-12-16 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30] [2026-01-18 00:00:00, 2023-12-17 00:00:00, 2027-01-16 00:00:00, 2024-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2027-01-09 00:00:00] [] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00] [2025-06-18 00:00:00, 2014-08-12 00:00:00, 2023-12-16 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00] [] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59, 2023-12-13 00:00:00, 2026-02-18 00:00:00, 2026-02-18 00:00:00, 2010-11-10 00:00:00, 2023-12-20 00:00:00] +279 2147483647 1198798030 -1 1 2023-12-16 2024-08-03 2023-01-15 2023-12-14 \N o 老乡卫生容量今年 这中间 好的咱们 诚心 南天安排新款动作考核代理 \N 1 40.123000 300.343000 -1.200000 1.200000 2025-02-18 00:00:00 2023-12-19 00:00:00 2024-02-18 00:00:00 2024-01-31 00:00:00 2023-12-14 00:00:00 2024-01-08 00:00:00 9999-12-31 23:59:59 2024-01-17 00:00:00 \N [1, 0, 0, 0, 1, 1, 1, 1, 1, 1] [0, 1, 0, 0, 1] [1, 0, 1, 1, 1, 0, 1, 0, 0, 0] [] \N [0, -24, -1, -128, 127] [-128, 127, 11, 0, 102] [-26631, -10592, 0, 32767, -10773, -14881, -1, -31052] [] [] [245, 21386, -519, -7399, 8903] [] [0, -1058557720, -1, -1693861056, 147483648, -2117598060, 0, -2001043051, 0, -206772322] [493298441, -2147483648, 0, 1918279187, -613940690, 1518826473, -2147483648, -1598514944, 1026641123, 147483648] [32679, 32679, -1, 147483648, 1659332459, -293412791, 32679, 527700280, 769241280, 0] [9223372036854775807, 245, 2132697756, 147483648, -7467131996044188183, 1, -5150877498461190986, 32679] [147483648, -9223372036854775808, -515212, -6352429, 147483648, -9223372036854775808, -8353813325976793346, 106896728] [5199314181433368039, 1, 9002025629144605314, 245, 178582, -9223372036854775808, 4140257177498810252, -9223372036854775808] [245, -5133414453454987448, 555772741, 5714806300478588641, 5832765, 147483648, 0, 1802477, -840750505, 3356815150016867175] [375499075, -1, 1, 0, 1, 1776962248, -2, -1] [-1, -51243933, 0, 0, 5, 1, -1, -2, 3, 0] [0, 1, -2, -2, 1, -2, -2, -696728758, -2, -1] [1, 1, -157621968, 0, -1, 0, 2, 1, -1370543407, 1] [88.148600000000, 93.120500000000, 19.137400000000, 24.026600000000, 100.020000000000, 0.000000000000, 41.162800000000, 40.123000000000, 300.343000000000, 100.020000000000] [300.343000000000, -1.200000000000, 88.076100000000, 18.166200000000, 87.153900000000, 59.045300000000, 63.092700000000, 1.200000000000, 96.039200000000, 98.102100000000] [0.000000000000, -1.200000000000, 1.200000000000, 0.000000000000, 1.200000000000, 1.200000000000, 16.186000000000, 0.000000000000, -1.200000000000, 40.123000000000] [100.020000000000, 0.000000000000, -1.200000000000, 49.079800000000, 0.000000000000, 0.000000000000, 1.200000000000, 100.020000000000] [] [1.2000, 55.1476, 37.0086, 13.0807, 1.2000, -1.2000, 40.1230, 91.1498] [30.1849, 62.1767, 40.1230, 300.3430, 0.0000, -1.2000, 87.0015, 1.2000] [40.1230, 1.2000, 40.1230, -1.2000, 0.0000, 100.0200, 300.3430, 80.0025, 40.1230, 100.0200] [40, -1, 1, 15, 64] [40, -1, 60, 1, -1, 300, -1, 100, 0, 0] [100, 69, 67, 40, -1, 7, 0, 100, 58, -1] [100, -1, 50] [] ["were look mean okay do go no", "I'll now have at", "-", "", "?", "for", "-", "-", "out some was that's about", "亿欧元陈总繁体字"] ["who he a got back her could", "would when in just don't you I'll here up", "", "卢海取款其人他在哪宇瑞", "think on", "who good just did there your yes had but", "here okay were you're did how", "what if"] ["降低", "", "he's", "-", "-", "had", "大概清楚时候最新版本肉麻考验", "?"] ["going", "", "he's if", "can", "-", "were of i I'll", "谁那里", "-"] ["who", "but have because can go", "was hey the can or go up", "?", "爱人一线内外青青绿色有带你催催", "大键盘", "-", "中环"] ["b", "--", "情愿", "t", "订货回报政策随风总经销商数量生活中", "we up out want going didn't but they", "", "because", "f", "-"] [] ["p", "y", "g"] [] ["q", "f", "l", "f", "g", "q", "g", "t"] ["y", "d", "s", "x", "y", "z", "m", "f"] [2023-12-16, 2024-07-01, 2027-01-16, 9999-12-31, 2027-01-16, 2014-08-12, 2025-06-18, 2024-02-18, 2025-02-18, 2026-02-18] [2023-12-11, 2024-01-17, 2023-12-20, 2024-08-03, 2027-01-09] [2025-06-18, 2025-06-18, 2023-12-09] [2023-12-20, 2023-12-13, 2025-02-17, 2023-12-20, 2024-02-18, 2025-06-18, 2011-10-24, 2025-06-18] \N [9999-12-31 00:00:00, 2023-12-15 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00, 2025-02-17 00:00:00, 2023-12-17 00:00:00, 2023-12-10 00:00:00, 2023-01-15 08:32:59] [2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30] [2023-12-10 00:00:00, 2026-02-18 00:00:00, 2025-06-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 2024-01-31 00:00:00] [2023-12-14 00:00:00, 2024-01-09 00:00:00, 2027-01-16 00:00:00, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2025-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-16 00:00:00, 2024-08-03 13:08:30, 2024-08-03 13:08:30] [] [2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2025-02-18 00:00:00] [2024-08-03 13:08:30, 2023-12-17 00:00:00, 2024-01-31 00:00:00] [2027-01-16 00:00:00, 2023-12-09 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2025-02-17 00:00:00, 2024-01-08 00:00:00] +280 680423406 32679 2147483647 -1176667468 2025-02-17 2025-06-18 2023-12-09 2023-12-15 \N 后面怡海那么大巴巴魏军随其自然南海乔总 - some got did I'm no is back so be at had how 1 0 \N 40.123000 40.123000 36.096900 2023-12-12 00:00:00 2024-06-30 12:01:02.123000 2026-02-18 00:00:00 2027-01-09 00:00:00 2025-06-18 00:00:00 2024-02-18 00:00:00 2024-02-18 00:00:00 2014-08-12 00:00:00 [] [0, 0, 1, 0, 1, 1, 0, 0, 1, 0] [] [0, 0, 1, 0, 1, 1, 0, 0, 0, 1] [127, 9, 127, -128, -128] [4, 8, -35] [0, -128, 70] [9, -128, 1, -128, 127, 6, -1, -127] \N [-32768, 1, 10211, 29013, -13514, 18295, 1, -1] [29689, 1, 32767] [-15019, 10069, 245, 32767, -1890, -13828, 245, -1] [147483648, -979784324, 147483648, 0, 32679] [] [32679, 147483648, 147483648] [420716596, 2028295660, 581855138, 1971003041, 2147483647] [-48573771, 2509248650262102574, 3336479696701330434, 1, 1, -5347271, 147483648, 3187410676903342648] \N [9223372036854775807, -5801515, -5573353389718569509, -6159364, 0, -391267, -6324061, -9223372036854775808, 1, -9223372036854775808] [32679, 32679, 245, 0, 8364617] [1, 2, 1, 8, 0, 1, 1440228330, 1] [0, 0, 0, 0, 1, -1, 1642599089, 5, -1, 0] [1, -1974146692, 1091406916, 6, 8, -1641653789, -1, 0, 1, 1] [-2, 1, 0, -1, 0, -2, -1, -2] [] [40.123000000000, 5.001800000000, 11.080400000000, 1.200000000000, -1.200000000000] [100.020000000000, 100.020000000000, 72.102000000000, 100.020000000000, 87.024000000000, 0.000000000000, 0.000000000000, 90.066200000000] [] [1.2000, 300.3430, 0.0000, -1.2000, 91.0647, 53.1471, 0.0000, -1.2000, 40.1230, 300.3430] [100.0200, 0.0000, -1.2000, 100.0200, 40.1230, 1.2000, 7.0258, -1.2000, 40.1230, 40.0350] [40.1230, 65.0728, 40.1230, 0.0000, 300.3430] [63.1253, 100.0200, 40.0500, 97.1794, 40.1230] [1, 38, 86] [64, 300, -1] [0, 100, 0, 5, -1, 100, 300, 86, 43, 33] [300, 38, 61, 300, 64] ["当作", "", "贵姓", "儿孙们", "百元整"] ["n", "我一会给大代理", "e"] ["", "some", "--"] ["-", "come", "well", "-", "?"] ["oh for oh with you're about she", "will", "?"] ["time", "-", "--", "me look all her got she were time like as", "who well yeah really tell when say"] [] ["很早", "-", "沃尔普解难先付聂柯赞美诗晨晖", "", "oh out of me can't would hey", "o", "hey", "?"] ["q", "w", "i", "a", "b"] ["z", "z", "j", "h", "f"] [] ["q", "p", "s", "y", "p"] [2025-06-18, 2023-12-12, 2027-01-09] [2023-12-13, 2023-12-16, 2024-02-18, 2024-01-31, 2023-12-10, 2025-02-18, 2023-12-15, 2025-06-18, 2025-02-18, 2023-12-20] [2025-02-18, 2023-12-09, 2023-12-09, 2014-08-12, 2024-01-08, 2025-02-17, 2024-01-08, 2023-12-11] [] [2024-08-03 13:08:30, 2014-08-12 00:00:00, 2014-08-12 00:00:00, 2024-07-01 00:00:00, 2025-02-18 00:00:00, 2023-01-15 08:32:59, 9999-12-31 23:59:59, 2023-12-19 00:00:00] [2026-01-18 00:00:00, 2023-12-18 00:00:00, 2023-12-14 00:00:00] [] [2023-12-12 00:00:00, 2014-08-12 00:00:00, 2023-12-09 00:00:00] [2023-12-17 00:00:00, 2023-12-10 00:00:00, 2014-08-12 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00] [2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] [2023-12-13 00:00:00, 2024-01-09 00:00:00, 2023-12-19 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 9999-12-31 00:00:00] [] [2024-02-18 00:00:00, 2014-08-12 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2023-01-15 08:32:59.123123, 2025-02-17 00:00:00] [2026-01-18 00:00:00, 2027-01-09 00:00:00, 2027-01-16 00:00:00, 2014-08-12 00:00:00, 2023-12-13 00:00:00, 9999-12-31 00:00:00, 2023-12-10 00:00:00, 9999-12-31 23:59:59, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [] [2023-12-10 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00] +281 1 32679 -1 1053923476 2025-06-18 2024-02-18 2024-01-19 2024-02-18 -- j 开会 赶紧 but we but going your really his how as who had good have all \N 1 100.020000 100.020000 300.343000 40.123000 2023-12-15 00:00:00 2023-12-14 00:00:00 2023-12-14 00:00:00 2024-08-03 13:08:30 2024-02-18 00:00:00 2024-01-17 00:00:00 2005-07-25 00:00:00 2023-12-14 00:00:00 \N [0, 1, 0, 0, 0, 0, 0, 0, 0, 0] [1, 0, 1, 0, 1] [1, 0, 1, 1, 1, 0, 0, 1, 0, 1] [1, 0, 8] [-21, -1, 1] [1, 10, 1, 1, 57, -21, 5, 2] [1, 127, 76, 0, 0, 2, 1, 127, -1, 7] [] [0, -7467, -11514] [1, 13000, 32767] [-32768, 22219, 14104, 0, -1, 19562, -12826, 3325, 245, -22926] [0, -324099026, -180627151, -891637222, 1220222513] [0, 1, 2147483647, -2147483648, -568657131, 653294370, 1490939388, 1, -2147483648, 909712460] [596907626, -581087303, -692979797, 1435803978, 32679, 2147483647, 2122105328, 2147483647] [32679, 819838999, 32679, -102061884, 0, -787159, -1, 1859916381] [6452156878165076592, 32679, 1, -1, -1] [32679, 0, -1540428295282082545, 8384031075621004259, -1, 0, 147483648, 7176599559210083208] [1285943255, 1, 1, -6152235426004416005, 3368236810938618613, 245, -9223372036854775808, 245, 0, 147483648] [0, 1910469938873153706, 32679] [1215745276, 1, 1860113209, -2, 0, 1, 1, 0, 9, -1] [-254240120, 217070540, 8] [5, -1, -2] [-2, -1363289075, -1] [40.123000000000, 300.343000000000, 40.123000000000, 300.343000000000, -1.200000000000, 59.162900000000, 53.003200000000, -1.200000000000, 53.129200000000, 22.012300000000] [39.165000000000, 0.000000000000, -1.200000000000, -1.200000000000, 0.000000000000] [300.343000000000, 40.123000000000, 40.123000000000, 100.020000000000, 60.144600000000, 300.343000000000, 40.123000000000, -1.200000000000] [300.343000000000, 95.036400000000, 51.080700000000, 0.000000000000, 100.020000000000] [0.0000, 1.2000, 0.0000, 300.3430, 300.3430] [65.1143, -1.2000, 58.0061, -1.2000, 96.0123, -1.2000, -1.2000, 1.2000, 47.0932, 300.3430] [3.1686, 100.0200, 84.1702, 300.3430, 1.2000, 3.1274, 40.1230, 300.3430] [300.3430, 0.0000, 1.2000, 16.0754, 300.3430] [72, 100, 40, 300, 1, 12, 300, 59, 40, 300] [77, 40, 40, 35, -1, 49, 82, 1] [1, -1, 66] [] [] ["几篇", "吵架", "又去", "from", "-"] [] ["模拟", "", "慈利达"] [] ["have", "看着办", "自私消息压在可是", "立博", "just", "would at", "-", "my going got she as time", "up", ""] ["叁仟", "e", "-", "j", "i", "you're oh it's him no she", "吃住", ""] ["time", "as", "is", "-", ""] ["n", "a", "w", "b", "k", "c", "t", "s", "g", "x"] ["e", "k", "i", "d", "c", "t", "f", "p", "q", "t"] ["n", "c", "l", "e", "t", "a", "y", "p", "m", "j"] ["d", "m", "g", "c", "r", "t", "u", "a", "f", "h"] [2027-01-16, 2023-12-10, 2025-02-17] [2023-12-13, 2023-12-13, 2024-02-18, 2026-01-18, 2024-01-17, 2023-12-11, 2024-01-31, 2024-08-03] [2014-08-12, 9999-12-31, 2024-02-18, 9999-12-31, 2023-12-09, 2023-12-14, 2024-01-31, 2023-12-15] [] [] \N [2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-19 00:00:00, 2023-12-13 00:00:00, 2023-12-13 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [2025-06-18 00:00:00, 2024-01-19 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00, 2024-01-08 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00] [2024-08-03 13:08:30, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 9999-12-31 00:00:00, 2023-12-16 00:00:00] [2024-01-31 00:00:00, 2023-12-16 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2024-08-03 13:08:30] [2024-02-18 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2024-01-31 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00] [2024-07-01 00:00:00, 2024-08-03 13:08:30, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00] [2025-06-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [2026-02-18 00:00:00, 2005-11-06 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2024-07-01 00:00:00, 2024-07-01 00:00:00, 2023-12-11 00:00:00, 2023-12-16 00:00:00] [2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00] [2024-01-17 00:00:00, 2025-02-18 00:00:00, 9999-12-31 00:00:00] +282 -1 2147483647 32679 1 2025-02-17 2023-01-15 2023-12-13 2027-01-09 表情含义爸妈没有办法最新代理不过 对策 - 冲动 昨晚上 明辉准的 不来 \N 1 -1.200000 75.025900 40.123000 300.343000 2025-02-17 00:00:00 2023-12-14 00:00:00 2014-08-12 00:00:00 2023-12-11 00:00:00 2026-01-18 00:00:00 2027-01-09 00:00:00 2025-06-18 00:00:00 2025-02-17 00:00:00 [0, 1, 0] \N [] [1, 0, 0, 1, 1, 1, 0, 0, 0, 0] [127, 1, -57, 1, 0, -74, 35, -99, 127, -1] [-66, -127, 127] [-1, 5, 4, 127, 1, 2, -70, -128] [] [-32768, 1, 32767, 1, 24421] [-17348, 245, 245, -12724, -10545, 32767, -32768, 245] [-31743, 245, 1, 2650, -13792, 32767, -21158, 1, 1, 1] [0, 25032, -21180, -1, 245] [-87729225, 32679, 1719236722, 2147483647, -1, 1, -1725645295, 1645571144, 2147483647, 1] [-2147483648, 1642674059, 32679, 1033658992, 1364332409] [1939674591, -1270954791, 2147483647, -516866696, 1200537016, -2147483648, -548473294, 0, -847896618, -587081334] [1097520629, 32679, 147483648, -536787682, -2147483648, -1, 1, 1, -2147483648, 2147483647] [7226850129418654896, -9223372036854775808, -9223372036854775808, 32679, -8524932202168148562] [245, 9223372036854775807, -1348034164, 9223372036854775807, 9223372036854775807] [245, 1, -1, 245, 6316334789950062997, 32679, -9223372036854775808, -1] [-7213083267430712017, 4959702, 147483648, 147483648, 4917239503696492787] [-2, 0, 0, 1, -1, 1, 1, -1] [3, 231670782, -2, 0, 2125670177] [] [1, -1, -2, -2, -1, 1, -1, 0, 0, -2] [-1.200000000000, 40.123000000000, -1.200000000000] [0.000000000000, 40.017800000000, 1.199400000000, 55.072500000000, 1.200000000000, -1.200000000000, -1.200000000000, 63.041800000000, 87.019900000000, 23.030600000000] [] [1.200000000000, -1.200000000000, 300.343000000000, 0.000000000000, -1.200000000000, 15.090800000000, 0.000000000000, -1.200000000000, 100.020000000000, 300.343000000000] [23.0676, 0.0000, 40.1230] [46.1754, 1.2000, 40.1230, 100.0200, 1.2000] [100.0200, 13.1273, 40.1230, 300.3430, 7.1474, 0.0000, 100.0200, 1.2000, 1.2000, -1.2000] [29.1765, 1.2000, -1.2000] [0, 100, 40, 100, 300, 1, -1, 27, 0, 48] [1, 1, 40, 29, 0, 73, 81, 100] [1, 0, 40] [] \N \N [] ["him did if is that not oh not want well", "?", "不想对你两个", "--", "?"] [] ["", "一百万关机功能规模乔总有你们是你呀外地稳定性通讯", "伤感", "相信", "王平"] ["just", "好了", "邮寄什么时间贰台防治如何个人"] ["did would i", "多好", ""] ["n", "y", "n"] ["h", "i", "n", "n", "n", "w", "p", "b"] [] ["r", "e", "r", "r", "g", "n", "d", "h"] [2023-12-20, 2024-01-17, 2024-08-03, 2019-07-11, 2023-12-18, 2024-02-18, 2023-12-18, 2025-06-18, 9999-12-31, 2024-02-18] [9999-12-31, 2025-06-18, 2009-01-04] [2024-01-19, 2026-01-18, 2027-01-09, 2023-12-13, 2023-12-12, 2024-02-18, 2023-12-12, 2024-02-18, 2023-12-09, 2026-02-18] [2023-12-20, 2025-06-18, 2023-12-17, 2024-08-03, 2023-12-09] [2027-01-09 00:00:00, 2023-12-14 00:00:00, 2023-12-10 00:00:00, 2027-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 9999-12-31 23:59:59] \N [] [2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2024-02-18 00:00:00, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2023-12-15 00:00:00] [2024-01-17 00:00:00, 2027-01-16 00:00:00, 2027-01-16 00:00:00] [] [] [2023-12-17 00:00:00, 2023-12-14 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2024-01-31 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2026-02-18 00:00:00, 2023-12-10 00:00:00] \N [2023-12-11 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2011-11-14 00:00:00, 2025-02-17 00:00:00, 2024-01-17 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00] [] [2023-12-16 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 2027-01-09 00:00:00] +284 32679 -2034582105 -413238173 0 2024-02-18 9999-12-31 2005-01-25 2024-07-01 场景不论意识配合预计型号身份证月亮五洲 okay in in do can't 那些 ? 安利 -- -- -- \N 0 7.188600 40.123000 -1.200000 100.020000 \N 2024-02-18 00:00:00 2024-01-08 00:00:00 2024-01-31 00:00:00 2023-12-12 00:00:00 2024-06-30 12:01:02.123000 2025-02-18 00:00:00 2024-01-09 00:00:00 [0, 0, 1, 1, 1, 1, 1, 0, 0, 1] [1, 0, 1, 0, 0, 0, 0, 1, 0, 0] [0, 0, 1, 0, 0] [] [92, 1, 121, -92, -127] [41, 70, 27, 1, -128, 127, -128, 0, -1, 127] [38, -128, 9, -9, 5] [54, -24, 101, 33, -1, 0, -1, 127, -93, 104] [15378, -7319, 29100, 1341, -27709] [-18072, 14721, 32767] [32767, -1, -32768, 245, 1] [] [-1, 1326582019, 1, 1, 1864360945] \N [] [426465914, 1673101477, -1] [-1, -722240820, -4129158919040803252, 32679, -1524664509820050366] [-1145132544094793184, -4282662628449939196, 245, 1, 0, 32679, 32679, 894468287, 8033472237110830948, -9223372036854775808] [0, 4291532433501819751, 32679, -1, 245] [] \N [6, 1, -1, 1, -1, 6, 1, 1] [] [552366393, 0, -1, 0, -1] [40.086800000000, -1.200000000000, 300.343000000000, 100.020000000000, 0.000000000000, 0.000000000000, -1.200000000000, 0.000000000000, 92.124600000000, 100.020000000000] [-1.200000000000, 0.000000000000, 85.109600000000, 1.200000000000, 1.200000000000] [0.000000000000, 33.038900000000, -1.200000000000] [] [1.2000, 40.1230, 87.1822, 43.0661, 100.0200, 1.2000, 100.0200, -1.2000] [65.1996, 0.0000, 1.2000, 19.1841, 43.0889, 300.3430, 22.1509, 300.3430, 14.0564, 300.3430] [] [40.1230, 40.1230, 40.1230] [100, -1, 0] [76, 1, 38, 56, 69, 40, 47, 89] [1, 0, 300, 34, 0] [40, 91, 100, 40, 300, 0, -1, 11, 1, 40] ["元月圈内签字高科这一生专用佳成上次了户确实", "-", "", "那也王海怎么样等级商丘杨学底下第二款", "--", "高密度", "from say good because I'm", ""] ["this going for to go don't in when come", "got", "-", "or got her and the do there his time", "?", "联硕", "for", "熊猫"] ["-", "him", "mean", "-", "", "can", "?", "-", "-", "well I'll or think i"] [] ["-", "can", "操心检查客服珍贵王丽网址主演本身核对", "三石", "?"] ["the you something up then time we", "yeah is she that ok I'm will as yes is", "x", "at he's got", "?", "time", "--", "or"] ["", "零度路上等会中环大企业普庆这三大承认高级总代理中原", ""] ["?", "一小时", "汉字", "then", "want ok all want and", "", "c", "--", "can do just as what mean a", "制度"] ["z", "b", "t", "z", "k"] ["l", "n", "j", "n", "p"] ["m", "v", "m", "n", "w", "z", "r", "o", "g", "f"] ["w", "i", "x"] [2023-12-17, 2025-06-18, 2023-12-10, 2010-09-18, 2023-12-13, 2024-06-30, 2025-06-18, 2024-02-18] [2014-08-12, 2023-12-13, 2025-02-17, 2023-12-10, 2024-06-30, 2024-01-31, 2024-06-30, 2024-07-01, 2027-01-16, 2024-08-03] [] [2023-12-15, 2025-02-17, 2024-02-18, 9999-12-31, 2024-01-17, 2024-02-18, 2023-12-19, 2025-02-18, 2023-12-17, 2023-01-15] [2023-12-14 00:00:00, 2023-12-16 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-01-19 00:00:00, 2023-12-17 00:00:00, 2024-01-17 00:00:00] [2026-02-18 00:00:00, 2019-08-27 00:00:00, 2023-12-13 00:00:00, 2024-01-09 00:00:00, 2025-02-18 00:00:00, 2023-01-15 08:32:59, 2023-12-20 00:00:00, 2024-06-30 12:01:02, 2023-12-09 00:00:00, 2026-02-18 00:00:00] [] [2025-06-18 00:00:00, 2025-02-17 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59, 2024-01-09 00:00:00] [2014-08-12 00:00:00, 2025-02-17 00:00:00, 2024-02-18 00:00:00] \N [2023-01-15 08:32:59.123000, 2023-12-13 00:00:00, 2023-12-09 00:00:00, 2023-01-15 08:32:59.123000, 2027-01-09 00:00:00] [] [2027-01-09 00:00:00, 2025-02-17 00:00:00, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2024-01-09 00:00:00] [2026-01-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00] [2025-06-18 00:00:00, 2023-12-09 00:00:00, 2023-12-19 00:00:00, 2024-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2027-01-16 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00] [] +286 1 1 147483648 1478631508 2023-12-12 2024-01-17 2025-06-18 2025-06-18 - \N 位居联想木马部门真实沉住气欣慰 to want that n - - 1 0 19.017000 57.096600 1.106000 100.020000 2023-12-11 00:00:00 2027-01-16 00:00:00 2023-12-13 00:00:00 2024-02-18 00:00:00 9999-12-31 00:00:00 2024-02-18 00:00:00 2023-12-19 00:00:00 9999-12-31 23:59:59 [1, 0, 1, 0, 0, 1, 1, 0] [0, 1, 0] [1, 1, 0] [1, 1, 1, 1, 1, 0, 1, 1] [17, 127, 0, 8, 1, -128, -44, 0, 127, 105] [] [-1, 15, 16, 0, 72, -128, 0, -18, 0, 1] [-1, 0, 60, 70, 96, 0, 1, 9] [10420, 32767, 1, 47, -2456] [19261, 17077, 12905, 30872, 31839, -6624, 24454, 18910, -13907, -7761] [31466, 6283, -32768, -20253, -9756] [] [2147483647, 813192517, 1424871367, 2147483647, 1639782204, 1067363591, 1, -244531365] [287242957, 0, 32679, -744835747, 297678538] [] [-55038871, -1, 0] [-203160261343895383, -1, -1, -1, 147483648, -9223372036854775808, 32679, 1] [32679, -8092731, 2326931090838173976, 8745855994674588811, -5921314] [-9223372036854775808, 2940695, 9223372036854775807] [-1, -1, 1505701966, -9223372036854775808, 1, 0, 8786838801246290524, -9223372036854775808] [1, 0, 1, 1, 1] [426337651, 0, 3, 1, 8, 0, -2, -1, -1, 1] [-2, 0, 0, 1, 2, 1, -2, 9] [] [19.157800000000, 0.000000000000, 40.123000000000, -1.200000000000, 72.086400000000, 1.200000000000, 78.191100000000, 100.020000000000] [1.200000000000, 8.083600000000, 52.149200000000, 17.053300000000, 0.000000000000] [12.176600000000, 79.017100000000, 40.123000000000, 300.343000000000, 100.020000000000, 9.196000000000, -1.200000000000, 77.197100000000] [100.020000000000, 80.132500000000, 100.020000000000, 1.200000000000, 40.123000000000] \N [84.1712, -1.2000, 100.0200, 40.1230, 77.0705] [-1.2000, 1.2000, 1.2000, 83.1657, -1.2000, 60.0895, 0.0000, 100.0200, 0.0000, 0.0000] [40.1230, 29.1958, 1.2000] [51, 16, 100] [82, 40, 0, -1, 1, 0, 0, 100, 91, 300] [] [7, 1, 2] ["删过", "are", "?", "?", "-", "your this you come", "--", "查到下次质保金防火墙石磊龙成主奴婢甘心中环来看"] ["利索", "总计", "-", "as", "one"] ["y", "I'm that a can't say because that", "?", "湖北", "he's"] ["电话本几台图标", "-", "?", "切换器", "okay", "适用于", "or", "as that", "--", "--"] [] [] ["been her mean he all to if in going get", "样品", "?", "?", ""] ["why that her with", "her", "", "怎么办", "我能"] ["z", "d", "b", "i", "d"] ["d", "i", "b", "k", "v", "q", "b", "a", "h", "b"] ["m", "z", "z", "d", "c", "k", "x", "i"] ["i", "y", "j", "o", "n", "s", "c", "k"] [2027-01-09, 2023-01-15, 2026-02-18] [2023-12-12, 9999-12-31, 2023-12-12, 9999-12-31, 2023-12-11, 2012-09-20, 2024-02-18, 2023-12-14] [2024-01-31, 2023-12-13, 2024-06-30] [2024-02-18, 2024-08-03, 2024-08-03, 2024-01-09, 2026-01-18] [2008-03-19 00:00:00, 2023-12-15 00:00:00, 2024-06-30 12:01:02] [2024-01-08 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2026-01-18 00:00:00, 2023-12-12 00:00:00, 2023-12-18 00:00:00, 2023-12-10 00:00:00, 9999-12-31 00:00:00] [2023-01-15 08:32:59, 2023-12-13 00:00:00, 2024-01-09 00:00:00] [2027-01-16 00:00:00, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00, 2026-01-18 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 9999-12-31 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00] [2014-08-12 00:00:00, 2023-12-19 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2007-06-23 00:00:00, 2026-01-18 00:00:00, 2023-12-19 00:00:00] [2024-01-31 00:00:00, 2023-12-18 00:00:00, 2027-01-16 00:00:00, 2023-12-13 00:00:00, 2023-12-18 00:00:00] [9999-12-31 00:00:00, 2024-01-19 00:00:00, 2023-12-15 00:00:00, 2023-12-13 00:00:00, 2024-01-19 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2023-12-09 00:00:00, 2026-02-18 00:00:00, 2026-01-18 00:00:00] [2023-12-11 00:00:00, 2023-12-15 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00] [2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00, 2023-12-17 00:00:00, 2023-12-16 00:00:00, 2023-12-10 00:00:00] \N [9999-12-31 23:59:59, 2024-01-19 00:00:00, 2024-06-30 12:01:02.123000] [2023-12-15 00:00:00, 2023-01-15 08:32:59.123123, 2024-08-03 13:08:30, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2024-01-19 00:00:00, 2023-12-20 00:00:00, 2025-02-17 00:00:00, 9999-12-31 23:59:59, 2023-12-19 00:00:00] +287 636501656 770360788 -2147483648 -870771891 9999-12-31 2024-08-03 2025-06-18 2023-12-11 畅谈 ? \N 淀粉技术参数地方而不是 伟泽 分区 ? that 1 1 \N 1.200000 300.343000 39.191200 2026-02-18 00:00:00 2024-02-18 00:00:00 2023-01-15 08:32:59.123000 2024-01-09 00:00:00 2023-12-20 00:00:00 2023-12-20 00:00:00 2024-01-19 00:00:00 2023-12-19 00:00:00 [1, 0, 1, 0, 1, 0, 0, 1] [1, 0, 1, 0, 1] [0, 1, 1, 0, 0, 0, 0, 0, 1, 1] [] [-1, -128, 3, -128, 0, 127, 0, -33] [-1, -56, -128] [-1, 96, 1, 0, 8] [-37, -128, 5, -1, 30] [-32768, -25059, 31225, 1424, 13424] [32767, -5335, 6200, -375, 245] [-17187, -27913, -21712, 24416, 32767] [-1, 19940, -18540, -1, -23499] [0, -523509783, 2147483647, 0, 32679, 1169379104, 710559770, -2147483648, 2147483647, 1661427213] [1, -573053238, 147483648, -1, 1131406956, 147483648, -605221494, -1] [2147483647, -396819009, -1, -745939356, 1147905213] [-2147483648, -529768583, 2147483647, 1096788310, -1818400842, -1609798898, -1, 147483648, -1792103040, 2147483647] [147483648, -3865732634171952561, -5962528493161325608] [77523, 147483648, 9223372036854775807, 32679, 0, -778221702679147773, -316427, 0] [147483648, 7667108, 0, -1, -9223372036854775808] [-1, -9223372036854775808, 32679, 4018471, 815127186, -1, -4049347323853095617, 0, 1229926255689452825, -1] [2, 7, 1] [1, 8, -2, -2, -1, 1, -2, 1, 9, -1141291371] [3, -453291536, 1, 52725630, -1957425975] [-1560667219, 8, 3, 1, -707461832] [61.181400000000, 12.099600000000, 40.123000000000, 0.000000000000, 17.046400000000, 3.182600000000, 47.008100000000, -1.200000000000] [] [1.200000000000, -1.200000000000, 26.153200000000, 12.043700000000, -1.200000000000, 0.000000000000, 300.343000000000, 90.069300000000] [300.343000000000, 37.037200000000, 40.123000000000] [] [31.0316, 40.1230, 0.0000, 40.1230, 300.3430] [300.3430, 1.2000, -1.2000, 0.0000, 48.1308, 0.0000, 40.1230, 1.2000, 31.0263, 100.0200] [] [] [0, 40, 300, 100, 300] [68, 100, -1, 300, -1, 99, 300, 300, 300, -1] [100, 300, -1, 100, 40, 0, 300, 100] ["?", "规模", "", "熊朝华三石有空吗留一下月份", "", "?", "--", "right", "盛源大家农村呵呵", "-"] \N [] ["你家", "就算别人现实中姚明", "come", "-", "礼貌全名淀雪五号", "no", "good", "--", "your back hey why like", "用在蝴蝶"] ["on don't would could for your do he's be", "I'm going on", "百家样机那个标书环保跟进中请你吃饭", "why", "", "yeah", "--", "-"] ["--", "参考一定从事有项目分销商杀客职务", "but got see", "正道", "", "that's me were can't was that", "商丘出国固定一改关闭研究院晕倒散落的出去需要", "短语", "could", "they"] ["想起来前几天返点金冠性质忽视那就算错回答", "-", "张先波备注"] [] ["z", "a", "e", "e", "o", "d", "t", "b", "u", "v"] ["m", "f", "h"] ["n", "f", "y"] ["o", "f", "p", "v", "r"] [2023-12-18, 2024-07-01, 2023-12-11, 2024-08-03, 2024-02-18, 2023-12-17, 2023-12-12, 2014-08-04] [9999-12-31, 2026-02-18, 2024-07-01, 2024-01-19, 2024-08-03] [2023-01-15, 2024-02-18, 2024-02-18, 2023-12-11, 2025-02-17, 2023-12-15, 9999-12-31, 2023-12-14, 2023-12-11, 2027-01-09] [2025-02-18, 2026-02-18, 2024-01-19, 2023-12-09, 2023-12-20, 2023-12-17, 2027-01-09, 2024-02-18, 2027-01-16, 2023-12-14] [2023-01-15 08:32:59, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-10 00:00:00] [2024-02-18 00:00:00, 2027-01-16 00:00:00, 2023-12-17 00:00:00] [2027-01-16 00:00:00, 2003-10-11 00:00:00, 2024-01-17 00:00:00, 9999-12-31 23:59:59, 9999-12-31 00:00:00, 2027-01-16 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2027-01-09 00:00:00] [2003-03-19 00:00:00, 2023-12-15 00:00:00, 2010-05-23 00:00:00, 2023-12-19 00:00:00, 2023-01-15 08:32:59, 2024-01-09 00:00:00, 2014-08-12 00:00:00, 2024-02-18 00:00:00] [2025-02-17 00:00:00, 2026-01-18 00:00:00, 2023-12-15 00:00:00] [] [2023-12-14 00:00:00, 2024-06-30 12:01:02.123000, 2001-04-16 00:00:00, 2024-08-03 13:08:30, 2023-01-15 08:32:59.123000, 9999-12-31 23:59:59, 2024-01-08 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2024-01-09 00:00:00] [2024-01-08 00:00:00, 2024-01-08 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2027-01-09 00:00:00, 2023-12-12 00:00:00, 2024-08-03 13:08:30, 2023-12-17 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000] [2014-08-12 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00] [2023-01-15 08:32:59.123123, 2023-12-11 00:00:00, 2023-12-19 00:00:00, 2023-12-13 00:00:00, 2024-06-30 12:01:02.123000, 2011-10-13 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00] [2023-12-17 00:00:00, 9999-12-31 00:00:00, 2023-12-09 00:00:00, 2023-01-15 08:32:59.123123, 2024-08-03 13:08:30, 2023-12-09 00:00:00, 2014-08-12 00:00:00, 2024-01-09 00:00:00, 2023-12-17 00:00:00, 2023-12-19 00:00:00] [] +288 790589626 -1137633788 -1 -1571939650 2023-12-20 2023-12-19 2023-12-13 2023-12-13 when come you all didn't just 玩玩特配机器过了毕业墙小王对方勇气全部答案 did would come how we you're tell the can yeah 常用汉字 now - that ? \N 1 -1.200000 100.020000 -1.200000 -1.200000 2024-06-30 12:01:02.123000 2023-12-11 00:00:00 2027-01-09 00:00:00 2024-02-18 00:00:00 2023-12-17 00:00:00 2023-12-17 00:00:00 2024-02-18 00:00:00 2026-01-18 00:00:00 [1, 0, 1, 1, 0] [1, 0, 0, 1, 1, 0, 0, 1] [0, 0, 0, 0, 1, 1, 1, 1] [1, 1, 0] [127, -1, 1, 127, 127, -1, 0, 9] [1, -67, -1, 0, 67, 127, 14, 1, 20, -128] [-128, 31, -128, 34, -128] [1, 5, -1, 2, -1] [-29445, -3570, -3627, 245, -3211] [2834, 26416, -715] [-25200, 0, 32767, 16556, -24295] [-25003, -10544, -32768, -1, 27343, -16821, -7218, 7685, -20993, -18673] [32679, 292541053, -624399042, 2147483647, -791643156, -1589787736, 0, 147483648] [2147483647, 20487165, -1010394079] [1, 0, 2147483647, -947021574, -220053974, -1, 1457273824, -1123059420] [-623299443, 2147483647, 1, 2147483647, 1, -1216957633, -2147483648, -1802824540, -1679851616, 701859137] [319997996, 0, 2739685817367483007, 1, 147483648, 147483648, 0, -1] [-1, 245, 0, 9223372036854775807, 0] [32679, -472123038, -2049150065609846920, 730348823, 0, -1, 32679, 9223372036854775807, -3807993973666535890, -6154624096791900438] [-9223372036854775808, -1, -420941719] [1, -1, 752042463, -2, 0] [-2, -1, -2, 690444110, 1004322797] [-1, 1, -1] [] \N [1.200000000000, 300.343000000000, 7.090600000000] [1.200000000000, 100.020000000000, 40.123000000000, 300.343000000000, 1.200000000000, 1.128100000000, 90.149200000000, 6.072600000000, 56.154300000000, 47.026400000000] [] [300.3430, 3.0851, -1.2000] [300.3430, 0.0000, 300.3430, 100.0200, 40.1230, 70.1226, 15.0032, 0.1910, 2.1842, 1.2000] [36.0399, 78.0996, -1.2000, 0.0000, 1.2000] [300.3430, 300.3430, -1.2000, 52.1929, 100.0200] [54, 89, 98, 99, -1, 300, 56, 0] [40, 39, 100, 54, 91, 40, 300, 76, 0, 300] [100, 100, 0] [1, 0, 300, 100, 300] \N ["羡慕要求核对过影视也冲击加我呼喊都说一贯", "不是顶多考前分销", "-", "-", "--", "怡海", "--", "以后公里", "?", "--"] ["那六新山行货服务器电池接受锦泽库存领出看是七匹狼", "打印机网通成本规定李景林网卡有些人就不均为", "could this didn't or I'll as can your if"] [] ["", "以后礼拜天总部焦作桌面愉快最大丁丁服务器上就要", "be here you're me"] ["really", "质保金老刘天翔漏洞生命中金蝶瑞恒销售量上核心总代理邮件", "--", "信息部够用好像住宿费报价合适规定电脑上拜拜套多", "m"] ["--", "第一首创任何赵燕燕帮助状态亚太一张一下", "okay here was", "and", "这几天"] [] ["q", "y", "x", "v", "f"] ["e", "d", "x", "x", "j", "n", "p", "d", "j", "s"] ["j", "e", "y", "o", "c"] ["k", "t", "t"] [2023-12-17, 9999-12-31, 2025-06-18] [2014-08-12, 2024-02-18, 2023-12-14] [2023-12-15, 2024-01-08, 2024-02-18, 2024-02-18, 2025-06-18] [2024-01-31, 2023-12-16, 9999-12-31, 2025-02-17, 9999-12-31, 2023-12-17, 2023-12-19, 2023-12-09, 2023-12-09, 2024-02-18] [] [2024-01-31 00:00:00, 2005-10-17 00:00:00, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00] [] \N \N [2023-12-14 00:00:00, 2027-01-09 00:00:00, 2027-01-09 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2024-01-31 00:00:00, 2023-12-09 00:00:00, 2025-02-18 00:00:00] [9999-12-31 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2016-03-22 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00] [2023-12-20 00:00:00, 2024-01-31 00:00:00, 2023-12-11 00:00:00] [2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00, 2023-12-12 00:00:00, 2024-01-17 00:00:00, 2023-12-18 00:00:00, 2027-01-09 00:00:00, 2023-12-15 00:00:00] [2023-12-20 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-11 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 9999-12-31 23:59:59, 9999-12-31 23:59:59, 2023-12-19 00:00:00] [2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-09 00:00:00, 2027-01-16 00:00:00, 2023-12-11 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-10 00:00:00] +289 0 \N 1153728948 258379093 2025-06-18 2014-08-12 2024-06-30 2024-08-03 新世纪好好哦啊组装那样子调价哥们工程不见三润 一览表给你 as go oh 还要 - 水晶关闭博源 right that tell have got but some your or why well as something a don't him go okay come 1 1 \N -1.200000 19.124000 98.035700 2024-01-19 00:00:00 2024-01-09 00:00:00 2027-01-16 00:00:00 2023-12-17 00:00:00 2023-12-10 00:00:00 2023-12-16 00:00:00 2023-12-14 00:00:00 2023-12-09 00:00:00 [0, 0, 1, 1, 1, 1, 1, 0] [0, 0, 1, 0, 0, 1, 0, 0] [0, 1, 1, 0, 1, 1, 1, 1] [0, 1, 1, 1, 1] [-1, 45, 0] [1, -128, 0, 1, 0, 1, 5, 1, -128, 7] [62, 3, -107, 1, 0, -32, -1, 1] [] \N \N [12956, 32767, 0] [15030, 2260, 245, 32767, -6588, -32768, -27308, -1, 32767, 1] [-1, 1781399970, 130829473, -1786958320, 1594339247] [] [32679, -537434250, -2147483648, 1, 2147483647, 2147483647, 999963413, 147483648] [-1, 32679, -1299735273, 2147483647, 0, 32679, -1781242326, 2147483647, 32679, 1281445097] [-3713145144828648722, -1, -176314143, 1, 7414841601902859189, -1208748076, 245, 147483648, 9223372036854775807, 9223372036854775807] [-7861716170982430627, 147483648, 147483648] [0, 9223372036854775807, 7209720628171482438, 1, 2410817] [-8275009794652258103, -1, 9223372036854775807, 0, -9223372036854775808, -1, -9223372036854775808, 357136, -1, 1] [9, -993366095, -1] [627673750, 1316392716, 1, -612760748, 1] [6, 1633283941, 0, -1, -1, -1, 1, -2, -1, 1] [1, -547699272, -1195670977, -234019479, -1118002839, -681448271, -1, -2] [0.000000000000, 51.119800000000, 0.000000000000] [300.343000000000, 82.183900000000, 100.020000000000, 83.069400000000, -1.200000000000, 40.123000000000, 100.020000000000, 100.020000000000, 100.020000000000, 69.139500000000] [100.020000000000, 0.000000000000, 53.160500000000, 15.124000000000, 0.146900000000, 100.020000000000, 40.123000000000, -1.200000000000, 300.343000000000, 100.020000000000] [0.000000000000, 300.343000000000, 0.000000000000, 1.200000000000, 300.343000000000] \N [5.1386, -1.2000, 0.0000, 40.1230, 30.0898] [64.0039, 29.1124, 57.0800] [24.0806, 300.3430, 49.1380, 100.0200, 100.0200] [100, 1, 0, -1, 22] [300, -1, -1, 41, 100] [300, -1, 1, 40, 40, 86, 0, 300] [1, 1, 300] ["-", "小刘", "高一些女朋友微软张小莉走之前一周", "y", "s", "股份有限公司", "got one not they are like about it's", "u"] ["-", "收藏夹", "had", "who know about now can't because", "we", "谁那里找你深爱金汇通机器各个自学考试南风", "", "--"] ["", "物流", "-", "?", ""] [] ["文华应用小型机承诺", "like from be she", "his good to", "采购周保全客户类型伟旭公司的", "发吧准的做事残品", "?", "", "?"] ["hey time got really back see you're did yeah how", "默默", "--", "from", "大法师告诉大家官方日月丰加上"] ["-", "刘伟", "逝去武汉自动化瑞信", "中心那样子建议以下小楼听雨", "come"] ["?", "半天班下个安排发布龙成艺术红雷王之参考", "?", "j", "考核前两天水平", "why", "伟旭网上", "were be from"] ["h", "t", "k", "t", "a"] \N ["f", "f", "q", "t", "o"] ["o", "o", "r", "c", "x", "j", "r", "s"] [2023-12-19, 2023-12-09, 2026-02-18, 2023-12-17, 2024-01-08, 2014-08-12, 2023-01-15, 2025-02-17] [2023-01-15, 2023-12-09, 2023-12-18] [2026-02-18, 2024-01-09, 2023-01-15] [] [2023-12-15 00:00:00, 2024-01-09 00:00:00, 2024-08-03 13:08:30] [] [2026-01-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00] [2023-12-19 00:00:00, 2024-01-17 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00, 2014-08-12 00:00:00, 9999-12-31 23:59:59, 2026-02-18 00:00:00, 2026-02-18 00:00:00] [2023-12-11 00:00:00, 2026-02-18 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00, 2024-02-18 00:00:00] [2009-03-08 00:00:00, 2024-01-17 00:00:00, 2024-01-17 00:00:00, 2023-12-17 00:00:00, 2025-06-18 00:00:00, 2027-01-16 00:00:00, 2023-01-15 08:32:59.123000, 2023-12-19 00:00:00, 2024-01-08 00:00:00, 2025-06-18 00:00:00] [2025-06-18 00:00:00, 2023-12-11 00:00:00, 2023-01-15 08:32:59.123000, 2024-01-31 00:00:00, 2023-12-16 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2016-08-06 00:00:00, 2023-12-15 00:00:00] [2023-12-19 00:00:00, 2023-12-19 00:00:00, 2027-01-09 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00, 2024-06-30 12:01:02.123000, 2026-01-18 00:00:00, 2024-02-18 00:00:00, 2023-12-15 00:00:00] [9999-12-31 00:00:00, 2023-12-11 00:00:00, 2023-12-16 00:00:00] [2018-02-06 00:00:00, 2023-12-12 00:00:00, 9999-12-31 00:00:00] [2025-06-18 00:00:00, 2027-01-16 00:00:00, 2025-02-18 00:00:00] [9999-12-31 23:59:59, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2024-07-01 00:00:00, 2026-01-18 00:00:00] +290 2147483647 -1217651742 -720099944 -1 2024-01-08 2027-01-16 2025-02-17 2024-01-31 宝莱动作 尽量 ? 乐凯代称免费又将鉴别不知道 o she - because something well ok 1 1 1.200000 64.116100 15.059700 100.020000 2026-01-18 00:00:00 2023-12-11 00:00:00 9999-12-31 00:00:00 2025-06-18 00:00:00 2023-01-15 08:32:59.123123 2026-01-18 00:00:00 2023-12-19 00:00:00 2023-12-18 00:00:00 [1, 0, 1, 1, 0, 0, 1, 0] [0, 0, 0, 1, 0, 1, 0, 1, 0, 1] [1, 1, 1] [1, 0, 0] [127, -1, 126] [5, 9, -1] [-1, -1, 0, -1, -128, 75, 1, 118, -32, 1] [] [29799, 26258, -11151, 21354, -25895] [32767, 245, 32767] [] [-32768, 32767, 1] [1, -2147483648, -233129254, 2147483647, 2147483647, -2147483648, 297629376, 2147483647] [1346054314, -1081237577, 1, -380424311, 0] [-2061450113, -1422287456, 1, 233472718, 786783928] [-569981449, -1298558901, -2147483648, -257089582, 1] [-1, 0, -1, -1, -9223372036854775808, 0, 320995084385170042, 1] [] [32679, 1407735705, 147483648, 2059884169845397720, 9223372036854775807, -9223372036854775808, 147483648, -8515137809967101117] [3812842, -9223372036854775808, 9223372036854775807] [-1, 0, 7, 1, 0] [7, 0, 63819391, -2, 1, 638101970, 2, 0, -1, -1] [-1, 0, -1] [-1, -1, 1431730817] [-1.200000000000, 40.123000000000, 100.020000000000, 0.000000000000, 40.123000000000, 0.000000000000, 100.020000000000, 44.124100000000, 300.343000000000, 78.057100000000] [0.000000000000, 100.020000000000, 51.186100000000, 40.123000000000, 300.343000000000] [56.053800000000, 300.343000000000, 40.123000000000, 0.000000000000, 1.200000000000, 54.178300000000, 0.000000000000, 81.071400000000] [88.071300000000, 100.020000000000, 0.000000000000, 33.033100000000, -1.200000000000] [54.1750, -1.2000, 17.1830] [-1.2000, 100.0200, 1.2000] [28.0964, 14.0887, 100.0200, 1.2000, 300.3430, 16.1945, 0.0000, 81.0771] [23.0692, 300.3430, 1.2000] [-1, 95, 36, 0, 100, 42, 32, 1] [1, 0, 92] [-1, 40, 0, 12, 1, 300, -1, 1] [49, 1, 54, 64, 300, 36, 70, 0] ["", "on and yeah", "c", "明天美女人参单位授狗只有大兵", ""] ["?", "聊聊", "?", "", "it", "经过", "r", ""] [] ["as see think you're if come tell so", "his what", "不设置"] ["天翔", "这也是", "", "--", "mean", "y", "?", "on"] ["贰万院内一直哟啊愿望封库力度洛阳燕龙", "don't", "k"] ["-", "", "w", "--", "?", "--", "this my about have my didn't what", "手册到帐招商不全相见仪器照片有点哈皮核好", "i", "there"] [] ["c", "l", "a"] ["k", "l", "l", "h", "i", "f", "c", "t", "w", "b"] ["c", "k", "p", "r", "d", "s", "n", "w", "w", "f"] ["y", "j", "k", "f", "o"] [] [2023-12-17, 2024-06-30, 2023-12-15, 9999-12-31, 2026-02-18, 2025-06-18, 2024-06-30, 2024-07-01, 2024-07-01, 2025-06-18] [2023-12-17, 2024-01-31, 2025-02-18, 2024-01-31, 2026-01-18] [] [2024-07-01 00:00:00, 2026-02-18 00:00:00, 2024-08-03 13:08:30, 2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-01-15 08:32:59, 2024-06-30 12:01:02, 2027-01-09 00:00:00, 2023-12-14 00:00:00, 2024-02-18 00:00:00] [2023-12-19 00:00:00, 9999-12-31 23:59:59, 2010-11-24 00:00:00] [] [9999-12-31 23:59:59, 2023-12-09 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 2024-06-30 12:01:02, 2025-06-18 00:00:00] [2026-02-18 00:00:00, 2027-01-09 00:00:00, 2024-01-08 00:00:00, 2023-12-20 00:00:00, 2024-08-03 13:08:30] \N [2023-12-12 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-06-30 12:01:02.123000, 2023-12-17 00:00:00, 2023-12-15 00:00:00] [9999-12-31 00:00:00, 2025-06-18 00:00:00, 2025-02-18 00:00:00, 2023-01-15 08:32:59.123000, 9999-12-31 23:59:59] [2025-02-17 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-31 00:00:00, 2027-01-09 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-16 00:00:00] [] [2024-02-18 00:00:00, 9999-12-31 23:59:59, 2025-06-18 00:00:00, 2024-01-31 00:00:00, 2024-01-08 00:00:00] [2023-12-17 00:00:00, 2025-02-17 00:00:00, 2014-08-12 00:00:00] +291 -796054732 479470141 101612700 1539907239 2023-12-20 2023-01-15 2023-12-19 2027-01-09 权威 降低 舍得到帐焦作市共享神仙喜剧元整中天开出来 - - it it's tell 1 0 46.118500 1.200000 40.123000 0.000000 2024-01-17 00:00:00 2025-06-18 00:00:00 2024-08-03 13:08:30 2024-01-17 00:00:00 2023-12-18 00:00:00 2024-02-18 00:00:00 2024-01-09 00:00:00 2024-01-09 00:00:00 \N [1, 1, 1, 0, 1, 1, 0, 0] [0, 1, 1] [0, 1, 0, 0, 0, 1, 0, 1] [] [15, 2, 127] [127, 12, -56, -97, 23, 1, -92, -1] [65, -98, 25, 0, -1] [-32768, 17288, 28225, 17607, 16417] [32767, 16612, 1, -7831, -28313] [32767, -1, -27766, 23733, 245, -1, 5461, 0, 21955, 17207] [] [0, -1, 1] [] [-1, -2147483648, 2147483647, -659549603, 2147483647, 2147483647, -2147483648, 2147483647, -2147483648, 1264303988] [1246691332, -1, 32679, 1, 1551905932, 147483648, 494643748, -804707034] [-5217178661736370154, 9223372036854775807, 147483648, 32679, -9223372036854775808, -9223372036854775808, -458029993371323095, 4241726685873642331, 8839175793896564017, -9223372036854775808] [1, 0, -9223372036854775808, 245, -9223372036854775808, 245, 147483648, 32679] [7991068177835444822, -4253743016973639763, 1250548179762142269, 147483648, 147483648, 1888170639, 9223372036854775807, -1, -1, -1] [32679, 0, -3787023579176815478, 2951010, 1, 0, -8602371225226494790, -9223372036854775808, 5735940199328383535, 0] [1, 0, 0, 1016605425, 0] [0, 0, -2] [-1, -566446407, 0] [0, 7, 0, 5, 1266176621] [300.343000000000, 300.343000000000, 100.020000000000] [-1.200000000000, 80.053700000000, 40.123000000000, 58.021800000000, 41.115500000000, 40.123000000000, 1.200000000000, 100.020000000000] [] [100.020000000000, 300.343000000000, -1.200000000000, 0.000000000000, 81.142300000000, 1.200000000000, 33.089000000000, 1.200000000000] [94.1887, 300.3430, 1.2000, 37.1381, 100.0200, 0.0000, 7.1620, 52.1175] [40.1230, -1.2000, 80.1023] [300.3430, -1.2000, -1.2000, 66.1653, 75.0717, -1.2000, 0.0000, -1.2000] [40.1230, 40.1230, -1.2000, -1.2000, 1.2000, 100.0200, -1.2000, -1.2000] [81, 78, 91, 1, -1, 43, 100, 40, 18, -1] [] [100, 100, 97, 300, -1, 40, 300, 0, 59, 1] [69, 1, 1] ["because then oh can't because come her him of can't", "at", "中原降低授权好多贸易"] ["", "", "j", "--", "--", "广场红包爽快耳畔好久理光", "okay", "银盘掉新赵洪应前奏参数出差享受阿奎那嵩县领导"] ["--", "分别我在政治哦推推订票做不到无限", "hey your tell do your had get good didn't as"] ["my", "mean I'll all some hey that who", "d"] [] ["--", "c", "", "不全忙碌直销付出了想不到手续综合", "?", "okay", "?", "okay"] ["g", "前两天离开", "部分人参多久人参司军几前辈陈总出面", "--", "看来"] ["k", "海豚问了重新发上海", "?", "something well", "刘汉"] ["m", "u", "r", "p", "y", "x", "q", "f"] ["f", "b", "w", "m", "o", "a", "z", "l"] ["i", "y", "y", "q", "i", "o", "y", "c", "v", "m"] ["j", "v", "n", "p", "q", "s", "f", "v"] [2025-06-18, 2024-01-09, 2023-12-19, 2025-06-18, 2023-12-10] [2024-07-01, 2024-01-09, 2024-02-18, 2023-12-14, 2027-01-09, 2023-12-17, 2023-12-19, 2026-01-18, 2024-01-17, 2023-12-11] [2023-12-09, 2024-02-18, 2023-12-10] [2023-12-14, 2024-01-31, 2024-06-30] [2027-01-09 00:00:00, 2023-01-15 08:32:59, 2025-06-18 00:00:00] \N [] [] [2027-01-16 00:00:00, 2023-12-16 00:00:00, 2027-01-16 00:00:00] [] [2026-02-18 00:00:00, 2024-07-01 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-17 00:00:00] [2027-01-16 00:00:00, 2024-02-18 00:00:00, 2026-01-18 00:00:00] [2023-12-18 00:00:00, 2025-06-18 00:00:00, 2023-12-17 00:00:00, 2025-02-17 00:00:00, 2023-12-18 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59.123123, 2024-02-18 00:00:00, 2024-08-03 13:08:30, 2024-02-18 00:00:00] [2024-01-17 00:00:00, 2026-01-18 00:00:00, 2024-01-08 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] [] [2023-12-20 00:00:00, 2027-01-16 00:00:00, 2024-08-03 13:08:30, 2024-01-31 00:00:00, 2023-12-17 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2023-12-19 00:00:00] +292 1 123947127 -1844103519 0 2023-12-10 2025-06-18 2024-02-18 2024-02-18 申请新年不开王海年底浪潮给你惊喜 - -- \N 很想 going up got \N 1 41.025100 92.146900 100.020000 0.000000 2024-01-19 00:00:00 2025-02-17 00:00:00 2024-01-31 00:00:00 2023-12-14 00:00:00 2023-12-16 00:00:00 2023-12-11 00:00:00 2023-12-10 00:00:00 2023-12-16 00:00:00 [0, 1, 0, 1, 1, 1, 0, 0, 0, 0] [0, 0, 1, 1, 1, 1, 1, 0, 0, 0] [] [0, 0, 1, 1, 0, 0, 0, 0] [0, 62, 43, 0, -1, 127, 127, 70] [-41, 127, 0, -43, -1] [125, 1, 0] [-105, 7, -100, 111, 1, 127, 85, 1, 1, -44] [245, -9909, 0, 32767, -32768, -25758, -1, 1] [] [1, -3207, 20219, 0, 0, -32768, 1, 32767] [] \N [] [-1305679577, -1, 1500472480, -1, -778776830, -1, 2147483647, 147483648, 1971639791, -2147483648] [1608355924, 2046356864, -1687908734, -2147483648, 32679, -284575047, -2147483648, -632999427, -2135642401, 1] [-6467995522863770316, 0, 1795114510, 147483648, -879295320] [32679, -1, 1789207280368923998] [-1981506904, 9223372036854775807, 147483648, 147483648, 147483648, 32679, -911693798, 4061553, 32679, 1515185965] [-1, -7584325282702187876, 7294060989580949829, -1, -1322426946817683451, -3186207, -1, -3702419062934981019, -9223372036854775808, 9223372036854775807] [1, 4, 0, 8, -2019393292, -242590287, -1, 2] [4, 0, 2057866289, -1, 6, 1, 1, -478306880, 4, 1] [4, 2, -1] [0, 5, -1, 1, -2, 6, -1, 0, -1, -497620505] [-1.200000000000, 1.200000000000, 75.146000000000, 40.123000000000, 68.017300000000] [40.123000000000, 55.178500000000, 100.020000000000, 100.020000000000, 100.020000000000, -1.200000000000, -1.200000000000, 0.000000000000, 1.200000000000, 1.200000000000] [40.123000000000, 18.058100000000, 0.000000000000, 1.200000000000, 40.123000000000] [40.123000000000, 300.343000000000, 40.123000000000, 62.116900000000, 100.020000000000] [72.1532, 100.0200, -1.2000, 0.0000, 1.2000, 40.1230, 36.1928, -1.2000, 1.2000, 1.2000] [] [-1.2000, 100.0200, 14.0054] [31.0754, -1.2000, 97.1247, -1.2000, 300.3430, 40.1230, 0.0000, -1.2000, -1.2000, 35.0893] [40, -1, 300] [1, -1, 40, 90, 0, 1, 40, -1] [31, 1, 100, 98, 40, 1, 100, 40, 1, 21] [] ["the mean it about and up at", "张婧俐选择众人最近", "美丽不高伤和气长期行踪金牌总代理自学月内运行", "--", "标注"] ["", "没有办法", "mean"] ["can how he's going that from", "唯一的", "谢谢英特设和那六减掉去做安排好"] ["-", "", "容量老家人选本就数据带", "离谱", "-"] [] ["h", "--", "like can"] ["?", "上半年", "yeah see with yes well see yeah think want as", "打电话", "?", "-", "t", "地球人"] ["必须整理伴随拿到", "各位", "中午", "--", "you", "--", "-", "标书看法客户城市协作广告词红颜只因为联恒金辉"] ["p", "b", "c", "x", "e"] ["n", "i", "d", "v", "y", "j", "z", "o"] ["m", "u", "w", "v", "q", "m", "o", "h", "i", "p"] ["k", "q", "a", "f", "d", "r", "g", "e", "g", "u"] [2024-01-17, 2025-06-18, 2024-01-09] [2023-01-15, 2026-01-18, 2023-12-10] [2024-07-01, 2023-12-16, 2023-12-10, 2023-12-20, 2024-06-30] [2023-01-15, 2024-08-03, 2023-12-20, 2023-12-20, 2027-01-16, 2025-02-18, 2023-12-19, 2027-01-16, 9999-12-31, 2023-12-15] [2023-12-20 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [2017-05-20 00:00:00, 2024-01-17 00:00:00, 2024-07-01 00:00:00] [2013-05-19 00:00:00, 2026-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-11 00:00:00, 2014-08-12 00:00:00] [2026-02-18 00:00:00, 2023-12-14 00:00:00, 2023-12-14 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2024-07-01 00:00:00, 2024-01-17 00:00:00, 2023-12-20 00:00:00, 2023-12-20 00:00:00, 2024-07-01 00:00:00] [2026-02-18 00:00:00, 2024-01-31 00:00:00, 2026-01-18 00:00:00, 2023-12-18 00:00:00, 2024-06-30 12:01:02.123000, 2024-02-18 00:00:00, 2011-05-23 00:00:00, 2024-01-31 00:00:00, 2023-12-14 00:00:00, 2023-12-13 00:00:00] [2014-08-12 00:00:00, 2001-11-19 00:00:00, 2024-06-30 12:01:02.123000, 2024-08-03 13:08:30, 2023-12-11 00:00:00, 2024-01-19 00:00:00, 2027-01-16 00:00:00, 2024-07-01 00:00:00] [2024-02-18 00:00:00, 2024-01-17 00:00:00, 2023-12-19 00:00:00] [] [2023-12-19 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-10 00:00:00, 2023-12-17 00:00:00, 2023-12-13 00:00:00, 2014-07-17 00:00:00] \N [] [9999-12-31 23:59:59, 2025-02-17 00:00:00, 2024-01-19 00:00:00] +293 -1 1797309764 147483648 -2132364647 2024-07-01 2023-01-15 2023-01-15 2023-12-19 鑫海源 that say had really time ok mean - 一条实力天数听着 商用 短语去不了稍等 好评 0 0 85.020100 40.123000 40.123000 40.165300 2024-01-31 00:00:00 2024-01-08 00:00:00 2024-07-01 00:00:00 2024-01-31 00:00:00 2027-01-16 00:00:00 2024-01-19 00:00:00 2023-12-19 00:00:00 2023-12-12 00:00:00 [1, 1, 1] [1, 1, 1, 1, 0] [] [1, 1, 1, 1, 0, 1, 1, 0] [127, 1, 7] [-128, 113, 83, -128, 0, -128, 6, -1] [1, -1, -128, 4, 1, 0, 3, -128] [1, -128, 1, 8, -128, 55, -1, 9, 72, 1] [32767, 0, -5069, 1, 0, 6954, 245, 245] [-32768, 32767, -32768, -1, 31439, 1261, -29628, 9142] [] [4182, -4255, -10139, 245, 245] [32679, 2147483647, -2147483648] [] [-2147483648, 32679, 1] [] [0, -3124333960959757953, 1, -9223372036854775808, 4619903] [-1, 1363413829, 9223372036854775807, -4509492, 147483648, 30435224, 1, 1910958810] [6798814273021461541, 9223372036854775807, 9223372036854775807, 0, 22151207207190793, 4629189667863036983, 1, 1, 1, 147483648] [] \N \N [-1, -1, -2, -1, 8, -1, 766837153, -2, -1, -1] [] [92.156500000000, 0.000000000000, 0.000000000000] [300.343000000000, 300.343000000000, 40.123000000000, 0.000000000000, 58.055200000000, 22.125000000000, 40.123000000000, 300.343000000000, 0.000000000000, 100.020000000000] [-1.200000000000, 40.123000000000, 99.183100000000, 40.123000000000, 100.020000000000, 1.200000000000, 1.200000000000, 0.000000000000, 40.123000000000, 40.123000000000] [8.101300000000, -1.200000000000, 300.343000000000, 300.343000000000, 0.000000000000, 23.095300000000, -1.200000000000, 30.143800000000] [1.2000, 94.1450, 91.1133, 51.0433, 0.0000, 36.1833, 0.0000, 72.1558] [] [0.0000, 300.3430, 300.3430] [100.0716, 51.1452, 300.3430, 43.0423, 300.3430, 88.0814, 63.1748, 100.0200] [74, 1, 50] [300, -1, -1, 0, 0] [300, -1, -1, 1, -1, 300, 25, 78] [1, 300, 40] ["o", "艺术瑞恒增票林慧方连一份子", "-", "大姐到帐黑色上面", "l", "you", "离开一会科苑怎么样令我业绩弄个高一些有限公司", "going", "okay been see he's he's back have", "ok can really"] ["-", "比爱白", "-", "跟着", "would"] ["she yeah is been go the", "--", "that's about he's about got he's he's"] ["同事置业鸿锐提成点需求早点志强", "q", "him"] ["独立", "吉林动手脚你不信吗水货嘉运进一步原则发展好好哦啊好句", "and", "叫做开除恶评", "but", "--", "--", "", "there", "图片天下无贼贸易请教新品一鸣程序梦幻水族弟子"] \N ["部分再来一声付费发邮箱能早点铁路洛宁总监蓝天", "?", "联盛", "营销", "my"] ["简单", "责任", "because"] ["w", "z", "r", "b", "h", "n", "x", "o", "v", "c"] ["e", "i", "r"] ["r", "f", "n", "e", "k", "z", "j", "j"] ["y", "k", "g", "o", "g", "o", "i", "w", "h", "n"] [2023-12-17, 2024-01-08, 2027-01-16, 2023-12-12, 2024-01-31, 2024-02-18, 2023-12-10, 2023-12-10] [2023-12-16, 2023-12-09, 2023-12-15] [2024-01-17, 2025-02-18, 2023-12-16, 2023-12-20, 2023-12-13, 2023-12-14, 2024-07-01, 2023-12-16, 2023-12-09, 9999-12-31] [2024-02-18, 2023-01-15, 2023-12-19] [2026-02-18 00:00:00, 2024-01-31 00:00:00, 2024-01-19 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2014-08-12 00:00:00, 2023-12-13 00:00:00] [2023-12-16 00:00:00, 2024-01-17 00:00:00, 2023-12-10 00:00:00, 2024-02-18 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00, 2023-12-10 00:00:00, 2025-02-18 00:00:00, 2023-12-13 00:00:00, 2023-12-09 00:00:00] [2002-04-02 00:00:00, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2024-02-18 00:00:00, 2023-12-14 00:00:00, 2026-02-18 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00] [2024-06-30 12:01:02, 2026-01-18 00:00:00, 9999-12-31 00:00:00, 2025-06-18 00:00:00, 9999-12-31 00:00:00] [2027-01-09 00:00:00, 2023-12-09 00:00:00, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 2025-02-17 00:00:00, 2026-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2024-07-01 00:00:00, 2024-01-31 00:00:00, 2024-02-18 00:00:00] \N [2024-02-18 00:00:00, 2023-01-15 08:32:59.123000, 2025-02-17 00:00:00, 2024-02-18 00:00:00, 2023-12-12 00:00:00, 2023-12-13 00:00:00, 2023-12-17 00:00:00, 2024-02-18 00:00:00] [2005-08-24 00:00:00, 2027-01-16 00:00:00, 2023-12-09 00:00:00] [2026-02-18 00:00:00, 9999-12-31 00:00:00, 2025-02-17 00:00:00, 2023-12-12 00:00:00, 2024-01-19 00:00:00] \N [2023-12-18 00:00:00, 9999-12-31 23:59:59, 2025-02-18 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00] [2024-01-19 00:00:00, 2024-01-09 00:00:00, 2027-01-09 00:00:00, 2023-12-13 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00, 2014-08-12 00:00:00, 9999-12-31 23:59:59] +294 0 1651076659 732954811 -2147483648 2025-06-18 2025-02-17 2023-12-16 2024-01-31 t 值得 家庭 \N he for come as hey one all come 在哪里呢 right now who can how right - \N 0 40.123000 40.123000 0.000000 40.123000 \N 2024-02-18 00:00:00 2027-01-16 00:00:00 2025-06-18 00:00:00 \N 2024-02-18 00:00:00 2024-02-18 00:00:00 2024-01-19 00:00:00 \N [0, 0, 1, 0, 0, 1, 1, 0, 0, 0] [0, 0, 0] [0, 0, 0, 0, 1] [-34, 0, 32, 1, -27] [-12, 2, 17] [-128, -128, -123, 108, 1] [-128, 127, 46, 127, 4, -128, -108, 1] \N [] [32767, 978, 31808, -3725, 0, -3299, 245, 21905] [31538, -10725, 32767, 245, 245] [0, 2147483647, 2147483647, -1, 147483648] [147483648, 261130839, 0] [1, 1479583589, -751048471, -1876362640, -1825168442] [] [-817932232842254010, 245, -9223372036854775808, -2552939242778840589, 1] [147483648, 2360755144017850001, 0, 32679, 245, -9044532933685131587, -1, -696319, -3980071236467434677, 1] [] [147483648, 6690986, 0, -9223372036854775808, 9223372036854775807] [] [-1, 0, 1, 1, 0] [0, 604733921, 0, -1, 0, 1, 0, 6] [-1612414618, -1587828192, 1, 3, -2, -1, 1, -1202484183] [0.000000000000, 40.123000000000, 48.125300000000, 62.002100000000, 300.343000000000, 1.200000000000, 40.123000000000, 0.000000000000] [72.079200000000, 300.343000000000, 100.020000000000] [] [0.000000000000, 13.187100000000, 1.200000000000, 40.123000000000, 4.086900000000, 40.123000000000, -1.200000000000, 0.000000000000] [] [-1.2000, 0.0000, 100.0200, 83.0581, 100.0200] [1.2000, 40.1230, 40.1230, 10.0954, 84.0822, 1.2000, 27.1340, 40.1230] [20.1611, 14.1182, 15.0528, -1.2000, 100.0200, 0.0000, 0.0000, 300.3430, 12.1567, 100.0200] [71, 40, 300, 100, 16] [0, 1, 300, 92, 0, -1, 36, 300] [92, 0, 51] [-1, 40, 78, 43, 40] ["-", "--", "is"] ["加载", "去谈", "近台南"] [] [] ["", "田辉", "--", "-", "didn't"] ["", "?", "?", "--", "法律区域一切开票决定", "安信", "as", "只因为", "--", "?"] ["--", "time", "北京写下中铁安利仁信肥姐俊峰小苏动态"] ["星月", "-", "about some the if out something me it's"] ["d", "y", "b", "o", "k"] \N ["g", "k", "d", "q", "j", "z", "e", "l"] ["i", "v", "a", "j", "c"] [2016-04-27, 2024-02-18, 2024-07-01] [2023-12-12, 9999-12-31, 2026-01-18, 2027-01-16, 2025-02-17, 2027-01-09, 2023-12-11, 2025-02-17, 2023-12-20, 2026-01-18] [2024-06-30, 2023-12-09, 9999-12-31, 2025-02-18, 2023-12-12] [] [] [9999-12-31 00:00:00, 2023-12-12 00:00:00, 2023-12-11 00:00:00, 2016-02-18 00:00:00, 2023-12-11 00:00:00] [2025-02-17 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 9999-12-31 23:59:59, 2024-01-09 00:00:00, 2024-08-03 13:08:30, 2023-12-18 00:00:00, 2025-06-18 00:00:00, 9999-12-31 23:59:59, 2023-12-11 00:00:00] [2023-12-20 00:00:00, 2024-01-31 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-12-18 00:00:00] [2023-12-16 00:00:00, 2023-12-10 00:00:00, 2023-12-11 00:00:00] [] [2027-01-09 00:00:00, 2024-02-18 00:00:00, 2024-02-18 00:00:00] [] \N \N [2023-12-14 00:00:00, 2024-06-30 12:01:02.123000, 2024-01-31 00:00:00] [] +297 32679 1833850362 1142590897 1 \N 2023-12-16 2023-12-20 2024-06-30 mean the well me in not be then look \N hey I'm you're to hey you up i had be one in -- did be 1 0 \N 1.200000 100.020000 49.133300 2003-12-15 00:00:00 2025-02-18 00:00:00 2023-12-15 00:00:00 2025-02-18 00:00:00 2024-01-08 00:00:00 2024-07-01 00:00:00 9999-12-31 00:00:00 2023-12-20 00:00:00 [0, 1, 1, 0, 1, 1, 0, 0] [1, 0, 1, 1, 1] [1, 1, 0, 0, 0] [] [] [-41, 1, -1] [] [8, 60, 0, 1, 6] [245, 15113, -21875] [458, 1, -8817, -23941, 1] [23712, -25052, -17363] [-10864, -2475, -15208, 14167, 4323] \N [-2147483648, 2147483647, -1936565406] [] [147483648, 147483648, 46598638, 1, 0, 147483648, 1, 147483648, 1, 1372374696] [9223372036854775807, 0, 0] [9072934316700164371, 0, 2807365, -7270424758999041321, 2700192, 9223372036854775807, -7504356290583361106, -1, 1, 32679] [32679, 647321271316391806, -1, 9223372036854775807, 32679, 147483648, -6840688708048815299, 4096410] [9223372036854775807, 1, 743284858338579503, 147483648, -6606384274549654296, -4789220, -9223372036854775808, 1] [-1, 0, -2, 0, 1] [1, -2, 1, -2, 4] [0, 1, 0, -2, 0] [-2, -2, 1] [0.000000000000, 100.020000000000, 8.099200000000, 83.130200000000, 32.149900000000, 6.168300000000, 94.103800000000, 0.000000000000] [75.168000000000, 95.117800000000, 100.020000000000] [40.123000000000, 44.086200000000, 57.197800000000, 0.000000000000, 1.200000000000, 84.145400000000, 100.020000000000, 0.000000000000, 15.119000000000, 40.123000000000] [39.051100000000, 300.343000000000, 0.000000000000] [0.0000, -1.2000, 0.0000, 300.3430, -1.2000, 74.1797, 40.1230, 1.1021, 0.0000, 58.1324] [300.3430, 66.1324, 40.1230, 40.1230, 40.1230, 0.0000, 40.1230, 300.3430] [66.0901, 1.2000, 100.0200, 1.2000, 40.1230] [] [1, 100, 40, 40, 1] [12, 100, 8] [] [] ["oh about don't my that there been he her something", "q", "v"] ["--", "here about got who", "也好", "失陪", "--"] ["when me her oh why about of", "容易", "up", "返点", "放假"] ["发过", "-", "h", "-", "吃的", "不信", "做方案农行王岩春福利道理询价金蝶提醒", "忙碌不但是送去"] ["", "--", "then go when could but okay"] \N ["还有", "something on", "my", "你吃法", "不及坐坐传美办款资源"] ["would", "", "武警总队计算", "y", "back back do yeah are right", "佳成代称个月", "-", "-", "good", "-"] ["g", "r", "c", "w", "k"] \N ["t", "w", "a", "b", "p", "s", "s", "s"] ["c", "c", "o"] [2026-01-18, 2026-01-18, 9999-12-31] [2024-02-18, 2026-02-18, 2023-12-19, 2023-12-17, 9999-12-31, 2026-01-18, 2023-12-10, 2005-08-20] [2027-01-09, 2025-06-18, 2024-01-31, 2023-12-14, 2023-12-17] [2024-01-19, 2025-06-18, 2024-01-19, 9999-12-31, 2024-02-18, 2024-01-17, 2024-02-18, 2023-12-20, 2025-02-18, 9999-12-31] [2024-01-08 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00] [2024-01-19 00:00:00, 2023-12-16 00:00:00, 2027-01-16 00:00:00, 2024-06-30 12:01:02, 2024-02-18 00:00:00, 2026-02-18 00:00:00, 2024-01-09 00:00:00, 2024-01-17 00:00:00, 2027-01-09 00:00:00, 2024-01-19 00:00:00] [2024-08-03 13:08:30, 2023-01-15 08:32:59, 2023-12-19 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00] [9999-12-31 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2023-12-12 00:00:00, 2023-12-09 00:00:00, 2023-12-16 00:00:00, 2026-01-18 00:00:00, 2024-08-03 13:08:30, 2024-01-19 00:00:00, 2023-12-13 00:00:00] \N [2023-12-10 00:00:00, 2024-07-01 00:00:00, 2024-01-09 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2027-01-09 00:00:00, 9999-12-31 23:59:59, 2024-08-03 13:08:30] [2023-12-20 00:00:00, 2023-12-17 00:00:00, 2023-12-20 00:00:00] [2023-12-17 00:00:00, 2006-06-02 00:00:00, 2014-08-12 00:00:00, 2024-01-17 00:00:00, 2025-06-18 00:00:00, 2026-02-18 00:00:00, 2024-06-30 12:01:02.123000, 2023-12-19 00:00:00, 2025-06-18 00:00:00, 2024-06-30 12:01:02.123000] \N [2024-01-19 00:00:00, 2018-04-04 00:00:00, 2011-05-23 00:00:00] [2023-12-12 00:00:00, 2023-12-15 00:00:00, 2024-02-18 00:00:00, 2024-01-19 00:00:00, 2024-01-09 00:00:00, 2023-12-15 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00] [2023-12-18 00:00:00, 2026-02-18 00:00:00, 2023-12-17 00:00:00, 2023-12-15 00:00:00, 2023-12-18 00:00:00] +298 \N -2147483648 1850652795 0 2026-01-18 2027-01-16 2024-08-03 2025-02-18 yes for with if know and his the on 任何 -- ? no think o \N 0 -1.200000 99.123100 100.020000 300.343000 2024-02-18 00:00:00 2024-08-03 13:08:30 2024-01-08 00:00:00 2023-12-09 00:00:00 2023-12-15 00:00:00 2024-01-31 00:00:00 2010-05-21 00:00:00 2024-01-19 00:00:00 [0, 1, 0, 0, 1] [1, 0, 0] [] [0, 1, 0, 0, 1, 1, 1, 0] [-1, -128, 127, 0, 127, 68, 8, -128, -48, 0] [-38, -128, 3, 127, 127, 7, -1, 51] [] [-128, 127, -3, 1, -82, 0, -1, -1] [1, 2706, 32767, -32768, 245, 25124, -32768, 32767, 21026, 19996] [245, 32767, -19710, 32767, -32768, -32332, 4028, 32275] [] [-32768, -32768, 0, -28136, 0, 245, 245, -32768] [-1, 360244913, 1072732548] [1, 147483648, -2147483648] [-2147483648, 0, 961470325, 2147483647, -1] [211173505, 32679, 1603426646, 0, -1, 32679, -2147483648, 657451891] [3855775670501303693, 9223372036854775807, 147483648, -185501413, 32679] [-5745265380401157142, -9223372036854775808, 147483648, 147483648, -1125647472086145602, 0, -9223372036854775808, -1, 0, 32679] [147483648, 0, 245, 1, 32679, 7832524, 32679, 147483648, 1, 7874905528790118777] [] [1, 6, -1, -2, 1] [-1, 0, 1, 563654093, -1252561266] [-2, -2, -1055907835] [0, 7, -2, 0, 0] [66.116100000000, 66.158800000000, 300.343000000000, -1.200000000000, -1.200000000000, 1.200000000000, 100.020000000000, 100.020000000000, 1.200000000000, 300.343000000000] [40.123000000000, 0.000000000000, 0.000000000000, 16.057000000000, 87.143200000000, 47.192500000000, 37.051000000000, 1.200000000000, 8.106400000000, 0.000000000000] [100.020000000000, 71.178600000000, 81.019300000000, 35.077900000000, 56.000300000000, 300.343000000000, 100.020000000000, 43.054500000000] [42.088000000000, 0.000000000000, -1.200000000000, 11.031300000000, 40.123000000000, 40.123000000000, 300.343000000000, 300.343000000000, 0.000000000000, 70.155900000000] [76.1208, 40.1230, 1.2000] [0.0000, 1.2000, 0.0000, 91.1162, 40.1230, 58.1763, 300.3430, 40.1230] [] [] [] \N [300, 13, 300, -1, 92] [] ["of", "b", "a", "p", "what what look oh want here", "?", "--", "老了世家你杀时间回来询问过直供等级硕博", "同志们", "最直接"] ["yes", "删除女人类型算数潍坊", "-", "总经销三门峡大河报张小燕追求一是也冲击分辨率祝你好运看中的", "really his did just i well time", "独显诚信晚上打还得", "really", "k", "y", "how for if"] ["快乐真理", "招生", "a", "my", "粘贴"] ["--", "剑派游戏合力看到宋玉慧交到陇海路", "that", "-", "电话本", "had", "出差灵生", "职务周伟燕刘广大忙人一条线硕博好句折叠", "say on that's", "at hey look why not who all this is just"] ["怎么样五笔不久指出来", "can right okay for no time because something see", "got", "get who some had not", "", "--", "-", "打架"] ["快乐", "he for don't this she", "when", "on", "--"] [] ["许愿树", "-", "美丽", "?", "系诶邪恶", "did", "集群", "here and"] ["u", "i", "i"] [] ["j", "l", "y"] ["o", "e", "u", "v", "w"] \N [2023-01-15, 2023-12-09, 2023-12-12] [2026-02-18, 2024-02-18, 2024-01-19] [2024-01-09, 2025-06-18, 2023-12-20, 2024-07-01, 9999-12-31, 9999-12-31, 2026-02-18, 2014-08-12, 2023-01-15, 2024-02-18] [2025-06-18 00:00:00, 2024-02-18 00:00:00, 2023-12-18 00:00:00, 2023-12-15 00:00:00, 2027-01-16 00:00:00] [2026-02-18 00:00:00, 2023-12-13 00:00:00, 2024-01-17 00:00:00, 2023-12-13 00:00:00, 2026-01-18 00:00:00, 2023-12-19 00:00:00, 9999-12-31 23:59:59, 2023-12-18 00:00:00, 2023-12-14 00:00:00, 2024-01-08 00:00:00] [] [] [2023-12-19 00:00:00, 2014-08-12 00:00:00, 2023-01-15 08:32:59.123000, 2026-01-18 00:00:00, 2024-01-09 00:00:00, 2024-02-18 00:00:00, 2023-12-09 00:00:00, 2023-12-09 00:00:00] [2024-01-08 00:00:00, 2024-01-08 00:00:00, 2024-07-01 00:00:00] [2023-12-14 00:00:00, 2025-06-18 00:00:00, 2027-01-09 00:00:00, 2023-01-15 08:32:59.123000, 2014-08-12 00:00:00, 2023-12-17 00:00:00, 2024-07-01 00:00:00, 2023-01-15 08:32:59.123000, 2024-02-18 00:00:00, 2024-07-01 00:00:00] [2024-07-01 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00] [2025-02-18 00:00:00, 2023-12-18 00:00:00, 2026-02-18 00:00:00, 2024-01-09 00:00:00, 2023-12-18 00:00:00, 2023-12-09 00:00:00, 2024-02-18 00:00:00, 2023-12-10 00:00:00] [2027-01-16 00:00:00, 2024-01-31 00:00:00, 2023-12-13 00:00:00, 2024-02-18 00:00:00, 2024-01-31 00:00:00, 2027-01-16 00:00:00, 2024-01-19 00:00:00, 2023-12-14 00:00:00] [2025-02-17 00:00:00, 9999-12-31 00:00:00, 2023-12-18 00:00:00, 2011-05-02 00:00:00, 2024-01-09 00:00:00, 2023-12-10 00:00:00, 2027-01-16 00:00:00, 2027-01-09 00:00:00, 2024-01-31 00:00:00, 9999-12-31 00:00:00] [2023-12-13 00:00:00, 2025-02-17 00:00:00, 2025-06-18 00:00:00, 2023-12-16 00:00:00, 2025-06-18 00:00:00, 2024-02-18 00:00:00, 2027-01-16 00:00:00, 2024-01-31 00:00:00, 2019-07-23 00:00:00, 2023-12-19 00:00:00] +299 147483648 879159405 73881529 450663434 2027-01-09 2023-12-15 2025-02-17 2023-12-11 how did well him me from and your i 产品事业部 really yes of 刘畅礼貌联系晨晖在吗也许自私小键盘 c 防火墙推磨认为 -- b 1 0 1.200000 40.123000 100.020000 24.033900 2024-02-18 00:00:00 2023-12-18 00:00:00 2024-01-09 00:00:00 2025-06-18 00:00:00 2024-02-18 00:00:00 2024-01-19 00:00:00 2024-02-18 00:00:00 2024-06-30 12:01:02.123000 [] [1, 0, 1, 0, 0, 0, 1, 1] [1, 0, 1, 1, 0, 0, 1, 1] [1, 0, 0, 0, 1, 0, 1, 1] [0, -128, -116] [-128, -24, 9, 0, 1, -12, 1, -109] [] [] [19800, 11369, 1] [-32768, 27000, 26735] [13535, -32768, 14020] [-21103, 0, -32768, -1, 1, -1, -1, -8909, -32768, -1] [-2147483648, -785207226, 1, -2098608653, 1547000442, -2147483648, -1762622263, -2147483648] [-1221995589, -2147483648, -1211720426, 1, 1139821263, 1, 147483648, 1533087374, 1842561401, 32679] [-1, 1526152989, 2147483647, 1, 2147483647, 0, 1, 147483648, -169054136, 147483648] [-1, -1696765043, -1, 1, 1359749547, -972238238, 32679, -739311908, -1, 0] [0, 1617706823318081005, -7940169234089772418, 245, -457846, 32679, -5775036928731241502, -7398278263486175984] [1, 1, 5379821326013865124, -9223372036854775808, 32679, -9223372036854775808, 245, -6057177913636272128, -1, 147483648] [3422229123326894841, 290645888, 3908526420334170936, 1, -1, 147483648, 1, 453336428711018078, -1, 32679] [-7019824124828882691, 147483648, 147483648] [-1, -2, -2] [0, -2, 1583980707, 0, -1, 0, -197368601, 193678458, 1, -1] [-2, 0, -1221889713, -2, -1394026358, 8, 200030701, -1] [-623695177, 0, 0, 1, 0, 1460288471, 9, -2] [40.123000000000, 300.343000000000, 1.200000000000, 0.000000000000, 300.343000000000, 69.044400000000, 91.080400000000, 0.000000000000, 0.000000000000, 0.000000000000] [-1.200000000000, 1.200000000000, 0.000000000000, 57.039000000000, 71.168400000000, -1.200000000000, 0.000000000000, 300.343000000000, -1.200000000000, 1.200000000000] [] [] [93.1058, 91.0943, 100.0200] [-1.2000, 100.0200, 44.0156, 30.0664, 15.1396, 89.1111, 38.0738, 100.0200] [0.0000, 100.0200, 100.0200, -1.2000, 14.0330] [39.1112, 0.0000, 0.0000, 40.1230, -1.2000] [] [40, 63, 77] [] [300, 0, 18, 74, 40, 90, 12, 59] ["点事不曾顺风配置下半年明基老大", "石龙", "中关村"] ["oh out didn't you his be mean what didn't how", "-", "some it were good her", "利达", "--"] ["回事发送改动可怕谢晓娟合作单位", "表格冲动房间联盛一夜周哥深圳这次去将来", "how"] ["", "right not they look but me as as with", "as because when will that's this ok", "?", "跨区了户"] ["弄个弄个才做", "-", "", "l", "or I'll be so going to are to they", "努力", "they something no", "?"] ["r", "那鱼", "", "自信", "偃师正版虽然"] ["", "r", "with out see and", "?", "玉冰说不定四川孙瑞霞那就好", "兰州", "", "or"] ["质保", "金辉", "-", "近台南", ""] ["d", "r", "u", "o", "x"] ["s", "i", "x", "v", "f", "b", "n", "t"] [] ["j", "d", "p"] [2023-12-18, 2025-02-18, 2008-07-01] \N [2024-01-19, 2024-02-18, 2024-02-18, 2024-01-19, 2023-12-14, 2025-06-18, 2026-02-18, 2014-08-12] [2024-01-19, 2024-07-01, 2023-12-11, 2023-12-17, 2024-01-19, 2025-06-18, 2026-01-18, 2025-02-18] [2024-01-08 00:00:00, 2023-12-18 00:00:00, 2024-07-01 00:00:00, 2024-06-30 12:01:02, 2024-01-08 00:00:00] [2025-02-18 00:00:00, 2025-06-18 00:00:00, 2024-01-17 00:00:00, 2023-12-09 00:00:00, 2023-12-20 00:00:00, 2023-12-10 00:00:00, 2009-08-15 00:00:00, 2014-08-12 00:00:00, 2025-06-18 00:00:00, 2023-12-11 00:00:00] [2024-01-08 00:00:00, 2023-12-19 00:00:00, 2024-01-31 00:00:00, 2024-07-01 00:00:00, 2024-02-18 00:00:00, 2014-08-12 00:00:00, 2023-12-19 00:00:00, 2025-06-18 00:00:00] [2024-01-09 00:00:00, 2023-12-12 00:00:00, 2024-01-17 00:00:00] [2014-08-12 00:00:00, 2024-01-19 00:00:00, 2024-01-31 00:00:00] [2024-06-30 12:01:02.123000, 2023-12-13 00:00:00, 2023-12-16 00:00:00, 2023-12-17 00:00:00, 2025-02-17 00:00:00, 2024-01-09 00:00:00, 2025-06-18 00:00:00, 2024-01-09 00:00:00, 2024-01-19 00:00:00, 9999-12-31 00:00:00] [2008-02-13 00:00:00, 2023-12-09 00:00:00, 9999-12-31 00:00:00, 2024-01-17 00:00:00, 2024-02-18 00:00:00, 2023-12-20 00:00:00, 2024-02-18 00:00:00, 2023-12-16 00:00:00] [2024-01-19 00:00:00, 2025-02-18 00:00:00, 2024-02-18 00:00:00, 2025-02-18 00:00:00, 2023-12-19 00:00:00] [2008-10-27 00:00:00, 2026-02-18 00:00:00, 2024-02-18 00:00:00] [2023-12-10 00:00:00, 2023-12-12 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00, 2024-01-19 00:00:00, 2024-01-19 00:00:00, 2023-12-12 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2024-02-18 00:00:00] [2024-02-18 00:00:00, 2023-12-20 00:00:00, 2025-06-18 00:00:00, 2023-01-15 08:32:59.123123, 2023-12-18 00:00:00] [2023-12-19 00:00:00, 2024-01-09 00:00:00, 9999-12-31 23:59:59, 2014-08-12 00:00:00, 2023-12-11 00:00:00, 2023-12-17 00:00:00, 2024-01-17 00:00:00, 2023-12-09 00:00:00, 2026-01-18 00:00:00, 2025-06-18 00:00:00] diff --git a/regression-test/data/nereids_p0/sql_functions/array_functions/test_array_filter.out b/regression-test/data/nereids_p0/sql_functions/array_functions/test_array_filter.out new file mode 100644 index 00000000000000..17c9732b9bca20 --- /dev/null +++ b/regression-test/data/nereids_p0/sql_functions/array_functions/test_array_filter.out @@ -0,0 +1,97 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +false \N +false \N +false \N +false \N +false \N +false \N +false \N +false \N +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +false true +true true +true true + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..4a84ff5117c008 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out @@ -0,0 +1,237 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +1 +1 +3 + +-- !groupby_pushdown_left_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_right_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_full_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_left_semi_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +1 3.0 +3 2.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +1 2 +1 3 +3 6 + +-- !groupby_pushdown_multi_table_join -- +1 +1 +1 + +-- !groupby_pushdown_with_order_by -- +1 +1 +1 +3 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +1 +1 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 1 +3 1 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 1 +c 1 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 1 1 +c 1 1 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 1.5 1 +1 4.5 1 +1 7.5 1 +3 7.0 0 + +-- !groupby_pushdown_with_order_by_limit -- +1 +1 +1 +3 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +1 +1 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +0 +1 +1 +1 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_left_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_right_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_full_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +1 3.0 +3 2.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +1 2 +1 3 +3 6 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +1 +1 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +1 +1 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 1 +3 1 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 1 +c 1 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 1 1 +c 1 1 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 1.5 1 +1 4.5 1 +1 7.5 1 +3 7.0 0 + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +1 +1 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_function_processed_columns -- +0 +1 +1 +1 + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index da69919becd7f2..8267eb3e38ff91 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -1034,3 +1034,25 @@ Used: UnUsed: use_push_down_agg_through_join_one_side SyntaxError: +-- !shape -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((dwd_tracking_sensor_init_tmp_ymd.dt = dw_user_b2c_tracking_info_tmp_ymd.dt) and (dwd_tracking_sensor_init_tmp_ymd.guid = dw_user_b2c_tracking_info_tmp_ymd.guid)) otherCondition=((dwd_tracking_sensor_init_tmp_ymd.dt >= substring(first_visit_time, 1, 10))) +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------filter((dwd_tracking_sensor_init_tmp_ymd.dt = '2024-08-19') and (dwd_tracking_sensor_init_tmp_ymd.tracking_type = 'click')) +------------------PhysicalOlapScan[dwd_tracking_sensor_init_tmp_ymd] +------------filter((dw_user_b2c_tracking_info_tmp_ymd.dt = '2024-08-19')) +--------------PhysicalOlapScan[dw_user_b2c_tracking_info_tmp_ymd] + +Hint log: +Used: use_PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE +UnUsed: +SyntaxError: + +-- !agg_pushed -- +2 是 2024-08-19 + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..db15483c496eb4 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out @@ -0,0 +1,237 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_right_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_full_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_semi_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +2 2.0 +3 3.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +1 6 +2 2 +3 3 + +-- !groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !groupby_pushdown_with_order_by -- +1 +1 +2 +3 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 1.5 a +1 7.0 \N +2 4.5 b +3 7.5 c + +-- !groupby_pushdown_with_order_by_limit -- +1 +1 +2 +3 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +\N +1 +1 +1 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_right_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_full_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +2 2.0 +3 3.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +1 6 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 1.5 a +1 7.0 \N +2 4.5 b +3 7.5 c + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_function_processed_columns -- +\N +1 +1 +1 + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..bb8366176a7ba6 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out @@ -0,0 +1,231 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +2 +3 +6 + +-- !groupby_pushdown_left_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_right_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_full_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_left_semi_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +2 2.0 +3 3.0 +6 2.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +2 2 +3 3 +6 6 + +-- !groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !groupby_pushdown_with_order_by -- +1 +2 +3 +6 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 +2 +3 +6 + +-- !groupby_pushdown_with_order_by_limit -- +1 +2 +3 +6 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +1 +2 +3 +6 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_left_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_right_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_full_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +2 2.0 +3 3.0 +6 2.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +2 2 +3 3 +6 6 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out b/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out new file mode 100644 index 00000000000000..32e6d2c32fb6d5 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.out @@ -0,0 +1,267 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !empty_tranform_not_to_scalar_agg -- + +-- !empty_tranform_multi_column -- + +-- !empty_tranform_multi_column -- +1 1 2 2 + +-- !tranform_to_scalar_agg_not_null_column -- + +-- !project_const -- +\N 1 +\N 1 +2 1 +2 1 +4 1 +6 1 +6 1 +10 1 + +-- !project_slot_uniform -- +2023-01-01 1 1 2023-01-01 + +-- !upper_refer -- + +-- !upper_refer_varchar_alias -- +cherry 3 + +-- !upper_refer_date -- +2023-01-06 + +-- !upper_refer_datetime_not_to_scalar_agg -- +2023-01-06T15:00 + +-- !upper_refer_datetime -- +2023-01-06T15:00 + +-- !project_no_other_agg_func -- +1 +1 +1 +1 +1 +1 +1 +1 + +-- !project_const_not_to_scalar_agg_multi -- +1 + +-- !not_to_scalar_agg_multi -- +1 1 2 2 + +-- !conflict_equal_value -- + +-- !project_slot_uniform_confict_value -- + +-- !inner_join_left_has_filter -- +100 100 + +-- !inner_join_right_has_filter -- +100 100 + +-- !left_join_right_has_filter -- +100 100 +101 \N +102 \N +103 \N +104 \N +105 \N +106 \N +107 \N + +-- !left_join_left_has_filter -- +100 100 +101 \N +102 \N +103 \N +104 \N +105 \N +106 \N +107 \N + +-- !right_join_right_has_filter -- +\N 101 +\N 102 +\N 103 +\N 104 +\N 105 +\N 106 +\N 107 +100 100 + +-- !right_join_left_has_filter -- +\N 101 +\N 102 +\N 103 +\N 104 +\N 105 +\N 106 +\N 107 +100 100 + +-- !left_semi_join_right_has_filter -- +100 + +-- !left_semi_join_left_has_filter -- +100 + +-- !left_anti_join_right_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !left_anti_join_left_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !left_anti_join_left_has_where_filter -- + +-- !right_semi_join_right_has_filter -- +100 + +-- !right_semi_join_left_has_filter -- +100 + +-- !right_anti_join_right_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !right_anti_join_left_has_on_filter -- +101 +102 +103 +104 +105 +106 +107 + +-- !right_anti_join_right_has_where_filter -- + +-- !cross_join_left_has_filter -- +100 100 +100 101 +100 102 +100 103 +100 104 +100 105 +100 106 +100 107 + +-- !cross_join_right_has_filter -- +100 100 +101 100 +102 100 +103 100 +104 100 +105 100 +106 100 +107 100 + +-- !union -- +1 100 +5 105 + +-- !union_all -- +1 100 +1 100 +5 105 + +-- !intersect -- + +-- !except -- + +-- !set_op_mixed -- +1 100 + +-- !window -- + +-- !partition_topn -- + +-- !cte_producer -- +1 1 100 + +-- !cte_multi_producer -- + +-- !cte_consumer -- + +-- !filter -- +1 100 + +-- !topn -- +1 100 + +-- !sink -- +\N 103 date 2023-01-04 2023-01-04T13:00 +\N 107 grape 2023-01-08 2023-01-08T17:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +1 100 apple 2023-01-01 2023-01-01T10:00 +2 101 banana 2023-01-02 2023-01-02T11:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +3 102 cherry 2023-01-03 2023-01-03T12:00 +4 104 elderberry 2023-01-05 2023-01-05T14:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +5 105 \N 2023-01-06 2023-01-06T15:00 +6 106 fig 2023-01-07 2023-01-07T16:00 + +-- !nest_exprid_replace -- +2023-10-17 2 2023-10-17 2 6 +2023-10-17 2 2023-10-18 2 6 +2023-10-17 2 2023-10-21 2 6 +2023-10-18 2 2023-10-17 2 6 +2023-10-18 2 2023-10-18 2 6 +2023-10-18 2 2023-10-21 2 6 + +-- !full_join_uniform_should_not_eliminate_group_by_key -- +\N 1 +105 1 + +-- !full2 -- +1 \N +1 105 + +-- !left_join_right_side_should_not_eliminate_group_by_key -- +\N 1 +105 1 + +-- !left_join_left_side_should_eliminate_group_by_key -- +\N 1 +105 1 + +-- !right_join_left_side_should_not_eliminate_group_by_key -- +1 \N +1 105 + +-- !right_join_right_side_should_eliminate_group_by_key -- +1 \N +1 105 + +-- !left_semi_left_side -- +1 +1 + +-- !left_anti_left_side -- +1 + +-- !right_semi_right_side -- +105 +105 + +-- !right_anti_right_side -- + diff --git a/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out b/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out index 3b9c3a1219ea15..c400e078daff1a 100644 --- a/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out +++ b/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out @@ -315,3 +315,35 @@ a 3 3 a,a,a 4.0 yy 3 1 a 4 2 a,a 4.0 yy 2 1 c 3 6 c,c,c 5.333333333333333 mi 3 2 +-- !query28_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query28_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query29_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query29_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query30_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query30_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query31_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query31_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + diff --git a/regression-test/data/nereids_rules_p0/mv/dml/rewrite/rewrite_when_dml.out b/regression-test/data/nereids_rules_p0/mv/dml/rewrite/rewrite_when_dml.out new file mode 100644 index 00000000000000..639b3b0b977a55 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/dml/rewrite/rewrite_when_dml.out @@ -0,0 +1,19 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- +o 3 21 o,o,o,o,o,o,o,o,o,o,o,o,o,o 4.571428571428571 mi 14 2 +o 4 4 o,o,o,o 4.0 yy 4 1 + +-- !query1_0_after -- +o 3 21 o,o,o,o,o,o,o,o,o,o,o,o,o,o 4.571428571428571 mi 14 2 +o 4 4 o,o,o,o 4.0 yy 4 1 + +-- !query2_0_before -- +o 3 21 o,o,o,o,o,o,o,o,o,o,o,o,o,o 4.571428571428571 mi 14 2 +o 4 4 o,o,o,o 4.0 yy 4 1 + +-- !query3_0_before -- +o 25 o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o 4.444444444444445 18 2 + +-- !query3_0_after -- +o 25 o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o 4.444444444444445 18 2 + diff --git a/regression-test/data/nereids_rules_p0/mv/micro_test/micro_test_when_cte.out b/regression-test/data/nereids_rules_p0/mv/micro_test/micro_test_when_cte.out new file mode 100644 index 00000000000000..8abaaf9adec0a8 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/micro_test/micro_test_when_cte.out @@ -0,0 +1,128 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query_0_after -- +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 + +-- !query_mv_0 -- +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 + +-- !query_mv_1 -- +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 + +-- !query_0_after -- +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 + +-- !query_mv_2 -- +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 + +-- !query_mv_3 -- +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-08 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-09 4 1 3 +2023-12-10 4 1 3 +2023-12-10 4 1 3 +2023-12-10 4 1 3 +2023-12-10 4 1 3 +2023-12-10 4 1 3 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-10 4 1 4 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-11 4 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 +2023-12-12 6 2 3 + diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out index f68b978b0b2ba6..2c44317371e8e1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query21.out @@ -11,16 +11,16 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[inv_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[inv_date_sk] ----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_date = '1999-05-23')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-07-22') and (date_dim.d_date >= '1999-05-23')) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out index 041e5711184598..61d2306286e791 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out @@ -14,17 +14,17 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_date = '2001-04-02')) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query21.out index db506f0acaa0e9..621490d6608ba7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query21.out @@ -21,6 +21,6 @@ PhysicalResultSink --------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ----------------------------PhysicalOlapScan[item] --------------------PhysicalProject -----------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) +----------------------filter((date_dim.d_date = '2002-01-28')) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out index 1465471817388b..a7af45b01ff19f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out @@ -25,6 +25,6 @@ PhysicalResultSink ------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------filter((date_dim.d_date = '2001-03-03')) ----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query21.out index 6a3b7ecf26ca2f..8fe10ba628d5f5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query21.out @@ -21,6 +21,6 @@ PhysicalResultSink --------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ----------------------------PhysicalOlapScan[item] --------------------PhysicalProject -----------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) +----------------------filter((date_dim.d_date = '2002-01-28')) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out index aae0d788557045..1fa5c0aabf528d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out @@ -25,6 +25,6 @@ PhysicalResultSink ------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------filter((date_dim.d_date = '2001-03-03')) ----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out index 991b448adf9f0c..6950418da2e44f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query21.out @@ -11,16 +11,16 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[inv_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[inv_date_sk] ----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_date = '2002-01-28')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out index ade38048fb9732..cc14224e1c7fe1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out @@ -14,17 +14,17 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_date = '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out index e80000c6353128..cb45325ac9e747 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out @@ -11,16 +11,16 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[inv_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[inv_date_sk] ----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_date = '2002-01-28')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out index 5ff27658e2ed3f..5c9d8fbfe4f88d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out @@ -14,17 +14,17 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_date = '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query21.out index 31448491385145..47437f6e6cc6fb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query21.out @@ -21,6 +21,6 @@ PhysicalResultSink --------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ----------------------------PhysicalOlapScan[item] --------------------PhysicalProject -----------------------filter((date_dim.d_date <= '1999-04-19') and (date_dim.d_date >= '1999-02-18')) +----------------------filter((date_dim.d_date = '1999-02-18')) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out index 414f6e3e0bd2ed..e7fac15753c030 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out @@ -25,6 +25,6 @@ PhysicalResultSink ------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2000-04-17') and (date_dim.d_date >= '2000-02-17')) +--------------------filter((date_dim.d_date = '2000-02-17')) ----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query21.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query21.out index db506f0acaa0e9..621490d6608ba7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query21.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query21.out @@ -21,6 +21,6 @@ PhysicalResultSink --------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ----------------------------PhysicalOlapScan[item] --------------------PhysicalProject -----------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) +----------------------filter((date_dim.d_date = '2002-01-28')) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out index 1465471817388b..a7af45b01ff19f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out @@ -25,6 +25,6 @@ PhysicalResultSink ------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------filter((date_dim.d_date = '2001-03-03')) ----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query21.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query21.out index 6a3b7ecf26ca2f..8fe10ba628d5f5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query21.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query21.out @@ -21,6 +21,6 @@ PhysicalResultSink --------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ----------------------------PhysicalOlapScan[item] --------------------PhysicalProject -----------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) +----------------------filter((date_dim.d_date = '2002-01-28')) ------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out index aae0d788557045..1fa5c0aabf528d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out @@ -25,6 +25,6 @@ PhysicalResultSink ------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) --------------------------PhysicalOlapScan[item] ------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------filter((date_dim.d_date = '2001-03-03')) ----------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query21.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query21.out index 991b448adf9f0c..6950418da2e44f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query21.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query21.out @@ -11,16 +11,16 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[inv_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[inv_date_sk] ----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_date = '2002-01-28')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out index ade38048fb9732..cc14224e1c7fe1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out @@ -14,17 +14,17 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_date = '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query21.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query21.out index e80000c6353128..cb45325ac9e747 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query21.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query21.out @@ -11,16 +11,16 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[inv_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[inv_date_sk] ----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_date = '2002-01-28')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '2002-03-29') and (date_dim.d_date >= '2002-01-28')) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out index 5ff27658e2ed3f..5c9d8fbfe4f88d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out @@ -14,17 +14,17 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_date = '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query21.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query21.out index f68b978b0b2ba6..2c44317371e8e1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query21.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query21.out @@ -11,16 +11,16 @@ PhysicalResultSink ----------------PhysicalProject ------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF2 w_warehouse_sk->[inv_warehouse_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[inv_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[inv_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = inventory.inv_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[inv_item_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((inventory.inv_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[inv_date_sk] ----------------------------PhysicalOlapScan[inventory] apply RFs: RF0 RF1 RF2 ----------------------------PhysicalProject -------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_date = '1999-05-23')) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= '1999-07-22') and (date_dim.d_date >= '1999-05-23')) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out index 041e5711184598..61d2306286e791 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out @@ -14,17 +14,17 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_date = '2001-04-02')) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ------------------PhysicalProject --------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/query_p0/aggregate/array_agg.out b/regression-test/data/query_p0/aggregate/array_agg.out index 1fe44df3a144bc..62ffb5fcf475c0 100644 --- a/regression-test/data/query_p0/aggregate/array_agg.out +++ b/regression-test/data/query_p0/aggregate/array_agg.out @@ -251,3 +251,6 @@ 8 [{"id":8}, {"id":8}, {"id":8}, {"id":8}] 9 [{"id":9}, {"id":9}, {"id":9}, {"id":9}] +-- !select -- +[null, "0.0.0.123", "0.0.12.42", "0.119.130.67"] [null, "::855d", "::0.4.221.183", "::a:7429:d0d6:6e08:9f5f"] + diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_trim_in.out b/regression-test/data/query_p0/sql_functions/string_functions/test_trim_in.out new file mode 100644 index 00000000000000..d62ae2744e3a12 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/string_functions/test_trim_in.out @@ -0,0 +1,547 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !empty_nullable -- + +-- !empty_not_nullable -- + +-- !all_null -- +\N +\N +\N + +-- !nullable -- +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N + +-- !not_nullable -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江省杭州市 + +-- !nullable -- + + + + ehlowrd +ac +余杭区 +西湖区 + +-- !not_nullable_null -- +\N +\N +\N +\N +\N +\N +\N + +-- !nullable_null -- +\N +\N +\N +\N +\N +\N +\N + +-- !const_nullable -- +\N +\N +\N +\N +\N +\N +\N + +-- !partial_const_nullable -- +\N +\N +\N +\N +\N +\N +\N + +-- !const_partial_nullable_no_null -- +ab +ab +ab +ab +ab +ab +ab + +-- !const_other_not_nullable -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江省杭州市 + +-- !const_not_nullable -- +abdc +abdc +abdc +abdc +abdc +abdc +abdc + +-- !1 -- + + +-- !2 -- + + +-- !3 -- +hello + +-- !4 -- +hello + +-- !5 -- +hello + +-- !6 -- +hello world + +-- !7 -- + + +-- !8 -- +hello + +-- !9 -- +hello + +-- !10 -- +hello + +-- !11 -- +hello world + +-- !12 -- + + +-- !13 -- + + +-- !14 -- + hello + +-- !15 -- + hello + +-- !16 -- +hello + +-- !17 -- + hello world + +-- !18 -- + + +-- !19 -- + hello + +-- !20 -- + hello + +-- !21 -- +hello + +-- !22 -- + hello world + +-- !23 -- + + +-- !24 -- + + +-- !25 -- + hello + +-- !26 -- +hello + +-- !27 -- +hello + +-- !28 -- +llo + +-- !29 -- +hello + +-- !30 -- +llo + +-- !31 -- +hello + +-- !32 -- +hello world + +-- !33 -- +llo world + +-- !34 -- + + +-- !35 -- +hello world + +-- !36 -- + + +-- !37 -- + + +-- !38 -- + hello + +-- !39 -- +hello + +-- !40 -- +llo + +-- !41 -- +hello + +-- !42 -- +llo + +-- !43 -- +hello + +-- !44 -- +hello world + +-- !45 -- +llo world + +-- !46 -- + + +-- !47 -- +hello world + +-- !48 -- + + +-- !49 -- + + +-- !50 -- + hello + +-- !51 -- + hello + +-- !52 -- + he + +-- !53 -- +hello + +-- !54 -- +he + +-- !55 -- +hello + +-- !56 -- + hello world + +-- !57 -- + hello wor + +-- !58 -- + + +-- !59 -- + hello world + +-- !60 -- +abc + +-- !61 -- + + +-- !62 -- + + +-- !63 -- + hello + +-- !64 -- + hello + +-- !65 -- + hello + +-- !66 -- + hello + +-- !67 -- + hello + +-- !68 -- +hello + +-- !69 -- + hello world + +-- !70 -- + hello world + +-- !71 -- + + +-- !72 -- + hello world + +-- !73 -- + +abcd +c浙江省杭州市a +llo +llo world +llo world +浙江省杭州市 + +-- !74 -- + +abcd +c浙江省杭州市a +llo +llo world +llo world +浙江省杭州市 + +-- !75 -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江省杭州市 + +-- !76 -- + + + +ac +lowrd +余杭区 +西湖区 + +-- !77 -- + + + +ac +lowrd +余杭区 +西湖区 + +-- !78 -- + + + + ehlowrd +ac +余杭区 +西湖区 + +-- !79 -- +\N +\N +\N +\N +\N +\N +\N + +-- !80 -- +\N +\N +\N +\N +\N +\N +\N + +-- !81 -- +\N +\N +\N +\N +\N +\N +\N + +-- !82 -- +\N +\N +\N +\N +\N +\N +\N + +-- !83 -- +\N +\N +\N +\N +\N +\N +\N + +-- !84 -- +\N +\N +\N +\N +\N +\N +\N + +-- !85 -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江 + +-- !86 -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江省杭州市 + +-- !87 -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江 + +-- !88 -- + + eh + ehlowrd +ac +he +区 +西湖区 + +-- !89 -- + + eh + ehlowrd +ac +he +区 +西湖区 + +-- !90 -- + + eh + ehlowrd +ac +he +余杭区 +西湖区 + +-- !91 -- + + hello + hello world + hello world +bcd +c浙江 +浙江 + +-- !92 -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江省杭州市 + +-- !93 -- + + hello + hello world + hello world +abcd +c浙江省杭州市a +浙江 + diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out index 19b192c95facb9..b672eb10570508 100644 --- a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.out @@ -35,3 +35,11 @@ doris9 3 888 888 30 4 40 40 40 +-- !sql -- +test1 15 +test2 29 +test3 49 + +-- !sql -- +3 + diff --git a/regression-test/data/variant_github_events_p2/affinityByIssuesAndPRs1.out b/regression-test/data/variant_github_events_p2/sql/affinityByIssuesAndPRs1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/affinityByIssuesAndPRs1.out rename to regression-test/data/variant_github_events_p2/sql/affinityByIssuesAndPRs1.out diff --git a/regression-test/data/variant_github_events_p2/affinityByIssuesAndPRs2.out b/regression-test/data/variant_github_events_p2/sql/affinityByIssuesAndPRs2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/affinityByIssuesAndPRs2.out rename to regression-test/data/variant_github_events_p2/sql/affinityByIssuesAndPRs2.out diff --git a/regression-test/data/variant_github_events_p2/authorsWithTheMostPushes.out b/regression-test/data/variant_github_events_p2/sql/authorsWithTheMostPushes.out similarity index 100% rename from regression-test/data/variant_github_events_p2/authorsWithTheMostPushes.out rename to regression-test/data/variant_github_events_p2/sql/authorsWithTheMostPushes.out diff --git a/regression-test/data/variant_github_events_p2/countingStar1.out b/regression-test/data/variant_github_events_p2/sql/countingStar1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/countingStar1.out rename to regression-test/data/variant_github_events_p2/sql/countingStar1.out diff --git a/regression-test/data/variant_github_events_p2/countingStar2.out b/regression-test/data/variant_github_events_p2/sql/countingStar2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/countingStar2.out rename to regression-test/data/variant_github_events_p2/sql/countingStar2.out diff --git a/regression-test/data/variant_github_events_p2/countingStar3.out b/regression-test/data/variant_github_events_p2/sql/countingStar3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/countingStar3.out rename to regression-test/data/variant_github_events_p2/sql/countingStar3.out diff --git a/regression-test/data/variant_github_events_p2/distributionOfRepositoriesByStarCount.out b/regression-test/data/variant_github_events_p2/sql/distributionOfRepositoriesByStarCount.out similarity index 100% rename from regression-test/data/variant_github_events_p2/distributionOfRepositoriesByStarCount.out rename to regression-test/data/variant_github_events_p2/sql/distributionOfRepositoriesByStarCount.out diff --git a/regression-test/data/variant_github_events_p2/githubRoulette.out b/regression-test/data/variant_github_events_p2/sql/githubRoulette.out similarity index 100% rename from regression-test/data/variant_github_events_p2/githubRoulette.out rename to regression-test/data/variant_github_events_p2/sql/githubRoulette.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears1.out b/regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears1.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears2.out b/regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears2.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears3.out b/regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears3.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears4.out b/regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears4.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears5.out b/regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears5.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears6.out b/regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears6.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.out diff --git a/regression-test/data/variant_github_events_p2/howHasTheTotalNumberOfStarsChangedOverTime.out b/regression-test/data/variant_github_events_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.out similarity index 100% rename from regression-test/data/variant_github_events_p2/howHasTheTotalNumberOfStarsChangedOverTime.out rename to regression-test/data/variant_github_events_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments1.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments1.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments1.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments2.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments2.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments2.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments3.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments3.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments3.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments4.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments4.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments4.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments4.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments5.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments5.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments5.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments5.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments6.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments6.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments6.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments6.out diff --git a/regression-test/data/variant_github_events_p2/issuesWithTheMostComments7.out b/regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments7.out similarity index 100% rename from regression-test/data/variant_github_events_p2/issuesWithTheMostComments7.out rename to regression-test/data/variant_github_events_p2/sql/issuesWithTheMostComments7.out diff --git a/regression-test/data/variant_github_events_p2/load.out b/regression-test/data/variant_github_events_p2/sql/load.out similarity index 100% rename from regression-test/data/variant_github_events_p2/load.out rename to regression-test/data/variant_github_events_p2/sql/load.out diff --git a/regression-test/data/variant_github_events_p2/mostForkedRepositories.out b/regression-test/data/variant_github_events_p2/sql/mostForkedRepositories.out similarity index 100% rename from regression-test/data/variant_github_events_p2/mostForkedRepositories.out rename to regression-test/data/variant_github_events_p2/sql/mostForkedRepositories.out diff --git a/regression-test/data/variant_github_events_p2/mostPopularCommentsOnGithub.out b/regression-test/data/variant_github_events_p2/sql/mostPopularCommentsOnGithub.out similarity index 100% rename from regression-test/data/variant_github_events_p2/mostPopularCommentsOnGithub.out rename to regression-test/data/variant_github_events_p2/sql/mostPopularCommentsOnGithub.out diff --git a/regression-test/data/variant_github_events_p2/organizationsByTheNumberOfRepositories.out b/regression-test/data/variant_github_events_p2/sql/organizationsByTheNumberOfRepositories.out similarity index 100% rename from regression-test/data/variant_github_events_p2/organizationsByTheNumberOfRepositories.out rename to regression-test/data/variant_github_events_p2/sql/organizationsByTheNumberOfRepositories.out diff --git a/regression-test/data/variant_github_events_p2/organizationsByTheNumberOfStars.out b/regression-test/data/variant_github_events_p2/sql/organizationsByTheNumberOfStars.out similarity index 100% rename from regression-test/data/variant_github_events_p2/organizationsByTheNumberOfStars.out rename to regression-test/data/variant_github_events_p2/sql/organizationsByTheNumberOfStars.out diff --git a/regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks1.out b/regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks1.out rename to regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks1.out diff --git a/regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks2.out b/regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks2.out rename to regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks2.out diff --git a/regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks3.out b/regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks3.out rename to regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks3.out diff --git a/regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks4.out b/regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks4.out similarity index 100% rename from regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks4.out rename to regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks4.out diff --git a/regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks5.out b/regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks5.out similarity index 100% rename from regression-test/data/variant_github_events_p2/proportionsBetweenStarsAndForks5.out rename to regression-test/data/variant_github_events_p2/sql/proportionsBetweenStarsAndForks5.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesByAmountOfModifiedCode.out b/regression-test/data/variant_github_events_p2/sql/repositoriesByAmountOfModifiedCode.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesByAmountOfModifiedCode.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesByAmountOfModifiedCode.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesByTheNumberOfPushes.out b/regression-test/data/variant_github_events_p2/sql/repositoriesByTheNumberOfPushes.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesByTheNumberOfPushes.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesByTheNumberOfPushes.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithClickhouse_related_comments1.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithClickhouse_related_comments1.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments1.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithClickhouse_related_comments2.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithClickhouse_related_comments2.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments2.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithDoris_related_comments1.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithDoris_related_comments1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithDoris_related_comments1.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithDoris_related_comments1.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithDoris_related_comments2.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithDoris_related_comments2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithDoris_related_comments2.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithDoris_related_comments2.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheHighestGrowthYoY.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheHighestGrowthYoY.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheHighestGrowthYoY.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheHighestGrowthYoY.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues1.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues1.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues2.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues2.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues3.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues3.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues4.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues4.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests1.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests1.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests2.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests2.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumNumberOfAcceptedInvitations.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMaximumNumberOfAcceptedInvitations.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess1.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess1.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay1.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay1.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay1.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheMostSteadyGrowthOverTime.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheMostSteadyGrowthOverTime.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.out diff --git a/regression-test/data/variant_github_events_p2/repositoriesWithTheWorstStagnation_order.out b/regression-test/data/variant_github_events_p2/sql/repositoriesWithTheWorstStagnation_order.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoriesWithTheWorstStagnation_order.out rename to regression-test/data/variant_github_events_p2/sql/repositoriesWithTheWorstStagnation_order.out diff --git a/regression-test/data/variant_github_events_p2/repositoryAffinityList1.out b/regression-test/data/variant_github_events_p2/sql/repositoryAffinityList1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoryAffinityList1.out rename to regression-test/data/variant_github_events_p2/sql/repositoryAffinityList1.out diff --git a/regression-test/data/variant_github_events_p2/repositoryAffinityList2.out b/regression-test/data/variant_github_events_p2/sql/repositoryAffinityList2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/repositoryAffinityList2.out rename to regression-test/data/variant_github_events_p2/sql/repositoryAffinityList2.out diff --git a/regression-test/data/variant_github_events_p2/starsFromHeavyGithubUsers1.out b/regression-test/data/variant_github_events_p2/sql/starsFromHeavyGithubUsers1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/starsFromHeavyGithubUsers1.out rename to regression-test/data/variant_github_events_p2/sql/starsFromHeavyGithubUsers1.out diff --git a/regression-test/data/variant_github_events_p2/starsFromHeavyGithubUsers2.out b/regression-test/data/variant_github_events_p2/sql/starsFromHeavyGithubUsers2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/starsFromHeavyGithubUsers2.out rename to regression-test/data/variant_github_events_p2/sql/starsFromHeavyGithubUsers2.out diff --git a/regression-test/data/variant_github_events_p2/theLongestRepositoryNames1.out b/regression-test/data/variant_github_events_p2/sql/theLongestRepositoryNames1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theLongestRepositoryNames1.out rename to regression-test/data/variant_github_events_p2/sql/theLongestRepositoryNames1.out diff --git a/regression-test/data/variant_github_events_p2/theLongestRepositoryNames2.out b/regression-test/data/variant_github_events_p2/sql/theLongestRepositoryNames2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theLongestRepositoryNames2.out rename to regression-test/data/variant_github_events_p2/sql/theLongestRepositoryNames2.out diff --git a/regression-test/data/variant_github_events_p2/theMostToughCodeReviews.out b/regression-test/data/variant_github_events_p2/sql/theMostToughCodeReviews.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theMostToughCodeReviews.out rename to regression-test/data/variant_github_events_p2/sql/theMostToughCodeReviews.out diff --git a/regression-test/data/variant_github_events_p2/theTotalNumberOfRepositoriesOnGithub.out b/regression-test/data/variant_github_events_p2/sql/theTotalNumberOfRepositoriesOnGithub.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theTotalNumberOfRepositoriesOnGithub.out rename to regression-test/data/variant_github_events_p2/sql/theTotalNumberOfRepositoriesOnGithub.out diff --git a/regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub1.out b/regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub1.out rename to regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub1.out diff --git a/regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub2.out b/regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub2.out rename to regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub2.out diff --git a/regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub3.out b/regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub3.out rename to regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub3.out diff --git a/regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub4.out b/regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub4.out similarity index 100% rename from regression-test/data/variant_github_events_p2/theTotalNumberOfUsersOnGithub4.out rename to regression-test/data/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub4.out diff --git a/regression-test/data/variant_github_events_p2/topRepositoriesByStars.out b/regression-test/data/variant_github_events_p2/sql/topRepositoriesByStars.out similarity index 100% rename from regression-test/data/variant_github_events_p2/topRepositoriesByStars.out rename to regression-test/data/variant_github_events_p2/sql/topRepositoriesByStars.out diff --git a/regression-test/data/variant_github_events_p2/whatIsTheBestDayOfTheWeekToCatchAStar.out b/regression-test/data/variant_github_events_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.out similarity index 100% rename from regression-test/data/variant_github_events_p2/whatIsTheBestDayOfTheWeekToCatchAStar.out rename to regression-test/data/variant_github_events_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.out diff --git a/regression-test/data/variant_github_events_p2/whoAreAllThosePeopleGivingStars1.out b/regression-test/data/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars1.out similarity index 100% rename from regression-test/data/variant_github_events_p2/whoAreAllThosePeopleGivingStars1.out rename to regression-test/data/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars1.out diff --git a/regression-test/data/variant_github_events_p2/whoAreAllThosePeopleGivingStars2.out b/regression-test/data/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars2.out similarity index 100% rename from regression-test/data/variant_github_events_p2/whoAreAllThosePeopleGivingStars2.out rename to regression-test/data/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars2.out diff --git a/regression-test/data/variant_github_events_p2/whoAreAllThosePeopleGivingStars3.out b/regression-test/data/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars3.out similarity index 100% rename from regression-test/data/variant_github_events_p2/whoAreAllThosePeopleGivingStars3.out rename to regression-test/data/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars3.out diff --git a/regression-test/framework/pom.xml b/regression-test/framework/pom.xml index 6b749bf0fd1dae..813659989ae60c 100644 --- a/regression-test/framework/pom.xml +++ b/regression-test/framework/pom.xml @@ -75,6 +75,7 @@ under the License. 4.9.3 2.8.0 1.11.95 + 17.0.0 diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy index 7b0d24482a29a1..f51e372a20d408 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy @@ -1027,19 +1027,23 @@ class Config { return buildUrlWithDb(jdbcUrl, dbName) } - Connection getConnectionByArrowFlightSql(String dbName) { + Connection getConnectionByArrowFlightSqlDbName(String dbName) { Class.forName("org.apache.arrow.driver.jdbc.ArrowFlightJdbcDriver") String arrowFlightSqlHost = otherConfigs.get("extArrowFlightSqlHost") String arrowFlightSqlPort = otherConfigs.get("extArrowFlightSqlPort") String arrowFlightSqlUrl = "jdbc:arrow-flight-sql://${arrowFlightSqlHost}:${arrowFlightSqlPort}" + "/?useServerPrepStmts=false&useSSL=false&useEncryption=false" - // TODO jdbc:arrow-flight-sql not support connect db - String dbUrl = buildUrlWithDbImpl(arrowFlightSqlUrl, dbName) + // Arrow 17.0.0-rc03 support jdbc:arrow-flight-sql connect db + // https://github.com/apache/arrow/issues/41947 + if (dbName?.trim()) { + arrowFlightSqlUrl = "jdbc:arrow-flight-sql://${arrowFlightSqlHost}:${arrowFlightSqlPort}" + + "/catalog=" + dbName + "?useServerPrepStmts=false&useSSL=false&useEncryption=false" + } tryCreateDbIfNotExist(dbName) - log.info("connect to ${dbUrl}".toString()) + log.info("connect to ${arrowFlightSqlUrl}".toString()) String arrowFlightSqlJdbcUser = otherConfigs.get("extArrowFlightSqlUser") String arrowFlightSqlJdbcPassword = otherConfigs.get("extArrowFlightSqlPassword") - return DriverManager.getConnection(dbUrl, arrowFlightSqlJdbcUser, arrowFlightSqlJdbcPassword) + return DriverManager.getConnection(arrowFlightSqlUrl, arrowFlightSqlJdbcUser, arrowFlightSqlJdbcPassword) } Connection getDownstreamConnection() { diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 650e949da8598b..cb220bffb903e9 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -732,6 +732,8 @@ class Suite implements GroovyInterceptable { runAction(new WaitForAction(context), actionSupplier) if (ObjectUtils.isNotEmpty(insertSql)){ sql insertSql + } else { + sql "SYNC" } if (cleanOperator==true){ if (ObjectUtils.isEmpty(tbName)) throw new RuntimeException("tbName cloud not be null") @@ -1814,6 +1816,22 @@ class Suite implements GroovyInterceptable { sql "analyze table ${db}.${mv_name} with sync;" } + def create_async_partition_mv = { db, mv_name, mv_sql, partition_col -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${db}.${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${db}.${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + PARTITION BY ${partition_col} + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + sql "analyze table ${db}.${mv_name} with sync;" + } + // mv not part in rewrite process void mv_not_part_in(query_sql, mv_name, sync_cbo_rewrite = enable_sync_mv_cost_based_rewrite()) { logger.info("query_sql = " + query_sql + ", mv_names = " + mv_name + ", sync_cbo_rewrite = " + sync_cbo_rewrite) @@ -1921,9 +1939,8 @@ class Suite implements GroovyInterceptable { check { result -> boolean success = true; for (String mv_name : mv_names) { - success = success && result.contains("${mv_name} chose") + Assert.assertEquals(true, result.contains("${mv_name} chose")) } - Assert.assertEquals(true, success) } } } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy index 856b0e76956395..e77658793fe5b2 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy @@ -23,12 +23,14 @@ import org.apache.doris.regression.util.JdbcUtils import org.apache.doris.regression.util.NodeType import com.google.common.collect.Maps +import org.awaitility.Awaitility import org.slf4j.Logger import org.slf4j.LoggerFactory import groovy.json.JsonSlurper import groovy.transform.CompileStatic import groovy.util.logging.Slf4j +import static java.util.concurrent.TimeUnit.SECONDS import java.util.stream.Collectors import java.sql.Connection @@ -333,7 +335,7 @@ class SuiteCluster { sqlModeNodeMgr = options.sqlModeNodeMgr - runCmd(cmd.join(' '), -1) + runCmd(cmd.join(' '), 180) // wait be report disk Thread.sleep(5000) @@ -483,6 +485,9 @@ class SuiteCluster { if (followerMode) { sb.append('--fe-follower' + ' ') } + if (sqlModeNodeMgr) { + sb.append('--sql-mode-node-mgr' + ' ') + } } if (beNum > 0) { sb.append('--add-be-num ' + beNum + ' ') @@ -492,7 +497,7 @@ class SuiteCluster { } sb.append('--wait-timeout 60') - def data = (Map>) runCmd(sb.toString(), -1) + def data = (Map>) runCmd(sb.toString(), 180) def newFrontends = (List) data.get('fe').get('add_list') def newBackends = (List) data.get('be').get('add_list') @@ -636,17 +641,15 @@ class SuiteCluster { } private Object runCmd(String cmd, int timeoutSecond = 60) throws Exception { - def fullCmd = String.format('python -W ignore %s %s --output-json', config.dorisComposePath, cmd) + def fullCmd = String.format('python -W ignore %s %s -v --output-json', config.dorisComposePath, cmd) logger.info('Run doris compose cmd: {}', fullCmd) def proc = fullCmd.execute() def outBuf = new StringBuilder() def errBuf = new StringBuilder() - proc.consumeProcessOutput(outBuf, errBuf) - if (timeoutSecond > 0) { - proc.waitForOrKill(timeoutSecond * 1000) - } else { - proc.waitFor() - } + Awaitility.await().atMost(timeoutSecond, SECONDS).until({ + proc.waitForProcessOutput(outBuf, errBuf) + return true + }) if (proc.exitValue() != 0) { throw new Exception(String.format('Exit value: %s != 0, stdout: %s, stderr: %s', proc.exitValue(), outBuf.toString(), errBuf.toString())) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy index d2d49efa1a86de..88ad1fac355d17 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy @@ -156,7 +156,7 @@ class SuiteContext implements Closeable { def threadConnInfo = threadArrowFlightSqlConn.get() if (threadConnInfo == null) { threadConnInfo = new ConnectionInfo() - threadConnInfo.conn = config.getConnectionByArrowFlightSql(dbName) + threadConnInfo.conn = config.getConnectionByArrowFlightSqlDbName(dbName) threadConnInfo.username = config.jdbcUser threadConnInfo.password = config.jdbcPassword threadArrowFlightSqlConn.set(threadConnInfo) diff --git a/regression-test/pipeline/external/conf/fe.conf b/regression-test/pipeline/external/conf/fe.conf index 92a6184fd92c75..2271abaa8e474e 100644 --- a/regression-test/pipeline/external/conf/fe.conf +++ b/regression-test/pipeline/external/conf/fe.conf @@ -25,13 +25,13 @@ LOG_DIR = ${DORIS_HOME}/log DATE = `date +%Y%m%d-%H%M%S` -JAVA_OPTS="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:+UseMembar -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xloggc:$DORIS_HOME/log/fe.gc.log.$DATE -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" +JAVA_OPTS="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:+UseMembar -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+PrintClassHistogramAfterFullGC -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xloggc:$DORIS_HOME/log/fe.gc.log.$DATE -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" # For jdk 9+, this JAVA_OPTS will be used as default JVM options JAVA_OPTS_FOR_JDK_9="-Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/fe.jmap -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=7 -XX:+CMSClassUnloadingEnabled -XX:-CMSParallelRemarkEnabled -XX:CMSInitiatingOccupancyFraction=80 -XX:SoftRefLRUPolicyMSPerMB=0 -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$DATE:time -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" # For jdk 17+, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*,classhisto*=trace:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" ## ## the lowercase properties are read by main program. diff --git a/regression-test/pipeline/p0/conf/fe.conf b/regression-test/pipeline/p0/conf/fe.conf index 48cc9598ef4b96..509446f9fcb0e3 100644 --- a/regression-test/pipeline/p0/conf/fe.conf +++ b/regression-test/pipeline/p0/conf/fe.conf @@ -27,10 +27,10 @@ CUR_DATE=`date +%Y%m%d-%H%M%S` LOG_DIR = ${DORIS_HOME}/log # For jdk 8 -JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" +JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+PrintClassHistogramAfterFullGC -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" # For jdk 17, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED" +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*,classhisto*=trace:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.ref=ALL-UNNAMED" sys_log_level = INFO sys_log_mode = NORMAL diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy index f6cea0e0a20497..4ee4d2221c6800 100644 --- a/regression-test/pipeline/p0/conf/regression-conf.groovy +++ b/regression-test/pipeline/p0/conf/regression-conf.groovy @@ -75,7 +75,6 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th "test_refresh_mtmv," + "test_spark_load," + "test_broker_load_func," + - "test_stream_stub_fault_injection," + "test_index_compaction_failure_injection," + "test_full_compaction_run_status," + "test_topn_fault_injection," + diff --git a/regression-test/pipeline/p1/conf/fe.conf b/regression-test/pipeline/p1/conf/fe.conf index 07a8b71b9a3ed1..168fe99c6e0790 100644 --- a/regression-test/pipeline/p1/conf/fe.conf +++ b/regression-test/pipeline/p1/conf/fe.conf @@ -27,10 +27,10 @@ CUR_DATE=`date +%Y%m%d-%H%M%S` LOG_DIR = ${DORIS_HOME}/log # For jdk 8 -JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" +JAVA_OPTS="-Djavax.security.auth.useSubjectCredsOnly=false -Xss4m -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError -XX:+UnlockExperimentalVMOptions -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDateStamps -XX:+PrintGCDetails -XX:+PrintClassHistogramAfterFullGC -Xloggc:$DORIS_HOME/log/fe.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Dlog4j2.formatMsgNoLookups=true -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" # For jdk 17, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" +JAVA_OPTS_FOR_JDK_17="-Djavax.security.auth.useSubjectCredsOnly=false -Xmx8192m -Xms8192m -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=$DORIS_HOME/log/ -Xlog:gc*,classhisto*=trace:$DORIS_HOME/log/fe.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Dcom.mysql.cj.disableAbandonedConnectionCleanup=true" sys_log_level = INFO sys_log_mode = NORMAL diff --git a/regression-test/suites/account_p0/test_information_schema.groovy b/regression-test/suites/account_p0/test_information_schema.groovy index dcbc0c3532838e..fc94ad4b9bc2fa 100644 --- a/regression-test/suites/account_p0/test_information_schema.groovy +++ b/regression-test/suites/account_p0/test_information_schema.groovy @@ -87,4 +87,26 @@ suite("test_information_schema") { def dbName = dbPrefix + i.toString() sql "DROP DATABASE `${dbName}`" } + + def dbName = dbPrefix + "default" + def tableName = tablePrefix + "default" + sql "CREATE DATABASE IF NOT EXISTS `${dbName}`" + sql "USE `${dbName}`" + sql """drop table if exists `${tableName}`""" + sql """ + CREATE TABLE `${tableName}` ( + `id` largeint NULL COMMENT '用户ID', + `name` varchar(20) NULL DEFAULT "无" COMMENT '用户姓名', + `age` smallint NULL DEFAULT "0" COMMENT '用户年龄', + `address` varchar(100) NULL DEFAULT "beijing" COMMENT '用户所在地区', + `date` datetime NULL DEFAULT "20240101" COMMENT '数据导入时间' + ) ENGINE=OLAP + DUPLICATE KEY(`id`, `name`) + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1") + """ + qt_default "SELECT COLUMN_NAME as field,COLUMN_TYPE as type,IS_NULLABLE as isNullable, COLUMN_DEFAULT as defaultValue FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = '${tableName}' AND TABLE_SCHEMA = '${dbName}'" + sql "DROP DATABASE `${dbName}`" } + diff --git a/regression-test/suites/account_p0/test_system_user.groovy b/regression-test/suites/account_p0/test_system_user.groovy new file mode 100644 index 00000000000000..1805f1669ea570 --- /dev/null +++ b/regression-test/suites/account_p0/test_system_user.groovy @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.junit.Assert; + +suite("test_system_user") { + test { + sql """ + create user `root`; + """ + exception "root" + } + test { + sql """ + drop user `root`; + """ + exception "system" + } + test { + sql """ + drop user `admin`; + """ + exception "system" + } +} diff --git a/regression-test/suites/auth_p0/test_select_view_auth.groovy b/regression-test/suites/auth_p0/test_select_view_auth.groovy new file mode 100644 index 00000000000000..87ec8cf0aeb758 --- /dev/null +++ b/regression-test/suites/auth_p0/test_select_view_auth.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("test_select_view_auth","p0,auth") { + String suiteName = "test_select_view_auth" + String user = "${suiteName}_user" + String pwd = 'C123_567p' + String dbName = "${suiteName}_db" + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String viewName = "${suiteName}_view" + + try_sql("drop user ${user}") + try_sql """drop table if exists ${dbName}.${tableName1}""" + try_sql """drop table if exists ${dbName}.${tableName2}""" + try_sql """drop view if exists ${dbName}.${viewName}""" + sql """drop database if exists ${dbName}""" + + sql """create user '${user}' IDENTIFIED by '${pwd}'""" + + //cloud-mode + if (isCloudMode()) { + def clusters = sql " SHOW CLUSTERS; " + assertTrue(!clusters.isEmpty()) + def validCluster = clusters[0][0] + sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; + } + sql """create database ${dbName}""" + sql("""use ${dbName}""") + sql """ + CREATE TABLE IF NOT EXISTS ${dbName}.`${tableName1}` ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE IF NOT EXISTS ${dbName}.`${tableName2}` ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """create view ${dbName}.${viewName} as select * from ${dbName}.${tableName1} union select * from ${dbName}.${tableName2};""" + + sql """grant select_priv on regression_test to ${user}""" + + // table column + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "select * from ${dbName}.${viewName}" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("denied")) + } + } + sql """grant select_priv on ${dbName}.${viewName} to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + sql "select * from ${dbName}.${viewName}" + } + + try_sql("drop user ${user}") + try_sql """drop table if exists ${dbName}.${tableName1}""" + try_sql """drop table if exists ${dbName}.${tableName2}""" + try_sql """drop view if exists ${dbName}.${viewName}""" + sql """drop database if exists ${dbName}""" +} diff --git a/regression-test/suites/bloom_filter_p0/test_bloom_filter.groovy b/regression-test/suites/bloom_filter_p0/test_bloom_filter.groovy index 89140593d0edf0..23e1c7ed596f62 100644 --- a/regression-test/suites/bloom_filter_p0/test_bloom_filter.groovy +++ b/regression-test/suites/bloom_filter_p0/test_bloom_filter.groovy @@ -115,4 +115,37 @@ suite("test_bloom_filter") { sql """ALTER TABLE ${test_map_tb} SET("bloom_filter_columns" = "k1,m1")""" exception "not supported in bloom filter index" } + + // bloom filter index for json column + def test_json_tb = "test_json_bloom_filter_tb" + sql """DROP TABLE IF EXISTS ${test_json_tb}""" + + test { + sql """CREATE TABLE IF NOT EXISTS ${test_json_tb} ( + `k1` int(11) NOT NULL, + `j1` json NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + DISTRIBUTED BY HASH(`k1`) BUCKETS 5 + PROPERTIES ( + "replication_num" = "1", + "bloom_filter_columns" = "k1,j1" + )""" + exception "not supported in bloom filter index" + } + + sql """CREATE TABLE IF NOT EXISTS ${test_json_tb} ( + `k1` int(11) NOT NULL, + `j1` json NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + DISTRIBUTED BY HASH(`k1`) BUCKETS 5 + PROPERTIES ( + "replication_num" = "1", + "bloom_filter_columns" = "k1" + )""" + test { + sql """ALTER TABLE ${test_json_tb} SET("bloom_filter_columns" = "k1,j1")""" + exception "not supported in bloom filter index" + } } diff --git a/regression-test/suites/check_before_quit/check_before_quit.groovy b/regression-test/suites/check_before_quit/check_before_quit.groovy index 23850a7c8a59f8..c9a71fe09aa865 100644 --- a/regression-test/suites/check_before_quit/check_before_quit.groovy +++ b/regression-test/suites/check_before_quit/check_before_quit.groovy @@ -226,5 +226,31 @@ suite("check_before_quit", "nonConcurrent,p0") { Thread.sleep(2000) } + + // check create table sql + List> allDataBases = sql "show databases;" + logger.info("show all databases: ${allDataBases}") + + def num = allDataBases.size() + + for (int i = 0; i < allDataBases.size(); i++) { + def db = allDataBases[i][0] + if (db == "__internal_schema" || db == "information_schema" || db == "mysql") { + continue + } + List> allTables = sql "show tables from ${db}" + logger.info("show all tabkes: ${allTables}") + for (int j = 0;j < allTables.size();j ++) { + def tbl = allTables[j][0] + def createTableSql = sql "show create table ${db}.${tbl}" + logger.info("create table sql info: ${createTableSql}") + sql "drop table if exists ${tbl}" + sql(createTableSql[0][1]) + def createTableSqlResult = sql "show create table ${tbl}" + logger.info("create table sql result info: ${createTableSqlResult}") + assertEquals(createTableSqlResult, createTableSql) + } + } + assertTrue(clear) } diff --git a/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy b/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy index 7405cb864d889d..70372f68ab865f 100644 --- a/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy +++ b/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy @@ -38,6 +38,7 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') { options.sqlModeNodeMgr = true options.waitTimeout = 0 options.feNum = 3 + options.useFollowersMode = true options.feConfigs += ["resource_not_ready_sleep_seconds=1", "heartbeat_interval_second=1",] } @@ -121,6 +122,9 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') { // Check FE number def frontendResult = sql_return_maparray """SHOW FRONTENDS;""" + // Check that all frontends are alive + def aliveCount = frontendResult.count { it['Alive'] == 'true' } + assert aliveCount == expectedFeNum, "Expected all $expectedFeNum frontends to be alive, but only ${aliveCount} are alive" assert frontendResult.size() == expectedFeNum, "Expected ${expectedFeNum} frontends, but got ${frontendResult.size()}" logger.info("FE number check passed: ${frontendResult.size()} FEs found") @@ -272,28 +276,23 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') { def feEditLogPort = feToDropMap['EditLogPort'] def feRole = feToDropMap['Role'] - logger.info("Dropping non-master frontend: {}:{}", feHost, feEditLogPort) + def dropFeInx = cluster.getFrontends().find { it.host == feHost }.index + logger.info("Dropping non-master frontend: {}:{}, index: {}", feHost, feEditLogPort, dropFeInx) // Drop the selected non-master frontend sql """ ALTER SYSTEM DROP ${feRole} "${feHost}:${feEditLogPort}"; """ - + // After drop feHost container will exit + cluster.dropFrontends(true, dropFeInx) + sleep(3 * 1000) + logger.info("Dropping frontend index: {}, remove it from docker compose", dropFeInx) // Wait for the frontend to be fully dropped - maxWaitSeconds = 300 - waited = 0 - while (waited < maxWaitSeconds) { + + dockerAwaitUntil(300) { reconnectFe() def currentFrontends = sql_return_maparray("SHOW FRONTENDS") - if (currentFrontends.size() == frontends.size() - 1) { - logger.info("Non-master frontend successfully dropped") - break - } - sleep(10000) - waited += 10 + currentFrontends.size() == frontends.size() - 1 } - if (waited >= maxWaitSeconds) { - throw new Exception("Timeout waiting for non-master frontend to be dropped") - } checkClusterStatus(2, 3, 4) @@ -309,86 +308,72 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') { } assert droppedFE != null, "Could not find the dropped frontend" - - feHost = droppedFE['Host'] - feEditLogPort = droppedFE['EditLogPort'] - - logger.info("Adding back frontend: {}:{}", feHost, feEditLogPort) - - // Add the frontend back - sql """ ALTER SYSTEM ADD FOLLOWER "${feHost}:${feEditLogPort}"; """ + + // Up a new follower fe and add to docker compose + // ATTN: in addFrontend, sql node mode, will execute `ALTER SYSTEM ADD FOLLOWER "${feHost}:${feEditLogPort}";` + boolean fuzzyUpFollower = (getRandomBoolean() == "true") ? true : false + logger.info("Want up a new role [{}] frontend", fuzzyUpFollower ? "FOLLOWER" : "OBSERVER") + def addList = cluster.addFrontend(1, fuzzyUpFollower) + logger.info("Up a new frontend, addList: {}", addList) + + def addFE = cluster.getFeByIndex(addList[0]) + feHost = addFE['Host'] + feEditLogPort = addFE['EditLogPort'] + def showFes = sql """SHOW FRONTENDS""" + logger.info("Adding back frontend: {}", showFes) // Wait for the frontend to be fully added back - maxWaitSeconds = 300 - waited = 0 - while (waited < maxWaitSeconds) { + dockerAwaitUntil(300, 5) { def updatedFrontends = sql_return_maparray("SHOW FRONTENDS") - if (updatedFrontends.size() == frontends.size()) { - logger.info("Frontend successfully added back") - break - } - sleep(10000) - waited += 10 + updatedFrontends.size() == frontends.size() } - - if (waited >= maxWaitSeconds) { - throw new Exception("Timeout waiting for frontend to be added back") - } - - // Verify cluster status after adding the frontend back + checkClusterStatus(3, 3, 5) logger.info("Frontend successfully added back and cluster status verified") // CASE 6. Drop frontend and add back again logger.info("Dropping frontend and adding back again") - // Get the frontend to be dropped - def frontendToDrop = frontends.find { it['Host'] == feHost && it['EditLogPort'] == feEditLogPort } + currentFrontends = sql_return_maparray("SHOW FRONTENDS") + + int obServerCount = currentFrontends.count { it['Role'] == 'OBSERVER' } + String fuzzyDropRole + if (obServerCount != 0) { + fuzzyDropRole = (getRandomBoolean() == "true") ? "FOLLOWER" : "OBSERVER" + } else { + fuzzyDropRole = "FOLLOWER" + } + + def frontendToDrop = currentFrontends.find {it['IsMaster'] == "false" && it['Role'] == fuzzyDropRole} + logger.info("Find drop again frontend: {}, drop role [{}]", frontendToDrop, fuzzyDropRole) assert frontendToDrop != null, "Could not find the frontend to drop" + def role = frontendToDrop.Role // Drop the frontend - sql """ ALTER SYSTEM DROP FOLLOWER "${feHost}:${feEditLogPort}"; """ - sleep(30000) + sql """ ALTER SYSTEM DROP $role "${frontendToDrop.Host}:${frontendToDrop.EditLogPort}"; """ + dropFeInx = cluster.getFrontends().find { it.host == frontendToDrop.Host }.index + // After drop frontendToDrop.Host container will exit + cluster.dropFrontends(true, dropFeInx) + logger.info("Dropping again frontend index: {}, remove it from docker compose", dropFeInx) + sleep(3 * 1000) reconnectFe() // Wait for the frontend to be fully dropped - maxWaitSeconds = 300 - waited = 0 - while (waited < maxWaitSeconds) { + dockerAwaitUntil(300, 5) { def updatedFrontends = sql_return_maparray("SHOW FRONTENDS") - if (!updatedFrontends.any { it['Host'] == feHost && it['EditLogPort'] == feEditLogPort }) { - logger.info("Frontend successfully dropped") - break - } - sleep(10000) - waited += 10 - } - - if (waited >= maxWaitSeconds) { - throw new Exception("Timeout waiting for frontend to be dropped") + !updatedFrontends.any { it['Host'] == frontendToDrop.Host && it['EditLogPort'] == frontendToDrop.EditLogPort } } - // Add the frontend back - sql """ ALTER SYSTEM ADD FOLLOWER "${feHost}:${feEditLogPort}"; """ + // Up a new follower fe and add to docker compose + // ATTN: in addFrontend, sql node mode, will execute `ALTER SYSTEM ADD FOLLOWER "${feHost}:${feEditLogPort}";` + addList = cluster.addFrontend(1, true) + logger.info("Up a new frontend, addList: {}", addList) - // Wait for the frontend to be fully added back - maxWaitSeconds = 300 - waited = 0 - while (waited < maxWaitSeconds) { + dockerAwaitUntil(300, 5) { def updatedFrontends = sql_return_maparray("SHOW FRONTENDS") - if (updatedFrontends.any { it['Host'] == feHost && it['EditLogPort'] == feEditLogPort }) { - logger.info("Frontend successfully added back") - break - } - sleep(10000) - waited += 10 + updatedFrontends.size() == 3 } - - if (waited >= maxWaitSeconds) { - throw new Exception("Timeout waiting for frontend to be added back") - } - // Verify cluster status after adding the frontend back checkClusterStatus(3, 3, 6) diff --git a/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy b/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy index 0a87c8d2704d1c..0940f817a8de27 100644 --- a/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy +++ b/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy @@ -20,7 +20,7 @@ suite("test_mv_case") { sql """CREATE TABLE `test_table_aaa2` ( `ordernum` varchar(65533) NOT NULL , `dnt` datetime NOT NULL , - `data` json NULL + `data` json NULL ) ENGINE=OLAP DUPLICATE KEY(`ordernum`, `dnt`) COMMENT 'OLAP' @@ -55,7 +55,7 @@ suite("test_mv_case") { createMV ("""create materialized view test_mv_view_t_mv as select `day`, count(game_code) from test_mv_view_t group by day;""") - sql """create view test_mv_view_t_view + sql """create view test_mv_view_t_view as select `day` from test_mv_view_t @@ -69,4 +69,69 @@ suite("test_mv_case") { sql("""SELECT * from test_mv_view_t_view where day='2024-04-15';""") notContains("mv_day") } + + sql """ drop table if exists tb1 """ + sql """ CREATE TABLE tb1 ( + `id` bigint NOT NULL COMMENT '', + `map_infos` map < int, + varchar(65533) > NULL COMMENT '' + ) ENGINE = OLAP UNIQUE KEY(`id`) COMMENT 'test' DISTRIBUTED BY HASH(`id`) BUCKETS 2 PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V1", + "enable_unique_key_merge_on_write" = "true", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728", + "enable_mow_light_delete" = "false" + ) + """ + sql """insert into tb1 select id,map_agg(a, b) from(select 123 id,3 a,'5' b union all select 123 id, 6 a, '8' b) aa group by id""" + createMV ("""CREATE MATERIALIZED VIEW mv1 BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 MINUTE DUPLICATE KEY(info_id) DISTRIBUTED BY HASH(`info_id`) BUCKETS 2 PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "colocate_with" = "dwd_info_group", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V1", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728", + "enable_nondeterministic_function" = "true" + ) AS + select + /*+ SET_VAR(enable_force_spill = true) */ + cast(a.id as bigint) info_id, + map_infos + from + tb1 a;""") + createMV ("""CREATE MATERIALIZED VIEW mv2 BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 MINUTE DUPLICATE KEY(info_id) DISTRIBUTED BY HASH(`info_id`) BUCKETS 2 PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "colocate_with" = "dwd_info_group", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V1", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728", + "enable_nondeterministic_function" = "true" + ) AS + select + info_id, + map_infos + from + mv1 a;""") + qt_select_mv """ select * from mv2 """ } diff --git a/regression-test/suites/export_p0/outfile/test_outfile_complex_type.groovy b/regression-test/suites/export_p0/outfile/test_outfile_complex_type.groovy new file mode 100644 index 00000000000000..49f81732791126 --- /dev/null +++ b/regression-test/suites/export_p0/outfile/test_outfile_complex_type.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("test_outfile_complex_type", "p0") { + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + def export_table_name = "test_outfile_complex_type_table" + def outFilePath = "${bucket}/outfile/complex_type/exp_" + + def outfile_to_S3 = { format -> + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${export_table_name} t + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS ${format} + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + + return res[0][3] + } + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE `${export_table_name}` ( + `dt` int(11) NULL COMMENT "", + `id` int(11) NULL COMMENT "", + `price` quantile_state QUANTILE_UNION NOT NULL COMMENT "", + `hll_t` hll hll_union, + `device_id` bitmap BITMAP_UNION + ) ENGINE=OLAP + AGGREGATE KEY(`dt`, `id`) + DISTRIBUTED BY HASH(`dt`) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + INSERT INTO `${export_table_name}` values + (20220201,0, to_quantile_state(1, 2048), hll_hash(1), to_bitmap(243)), + (20220201,1, to_quantile_state(-1, 2048), hll_hash(2), bitmap_from_array([1,2,3,4,5,434543])), + (20220201,2, to_quantile_state(0, 2048), hll_hash(3), to_bitmap(1234566)), + (20220201,3, to_quantile_state(1, 2048), hll_hash(4), to_bitmap(8888888888888)), + (20220201,4, to_quantile_state(2, 2048), hll_hash(5), to_bitmap(98392819412234)), + (20220201,5, to_quantile_state(3, 2048), hll_hash(6), to_bitmap(253234234)); + """ + + // parquet file format + def format = "parquet" + def outfile_url = outfile_to_S3("${format}") + qt_select_load_parquet """ SELECT dt, id, hex(price), hex(hll_t) FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // orc file foramt + format = "orc" + outfile_url = outfile_to_S3("${format}") + qt_select_load_orc """ SELECT dt, id, hex(price), hex(hll_t) FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // csv file foramt + format = "csv" + outfile_url = outfile_to_S3("${format}") + qt_select_load_csv """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ +} \ No newline at end of file diff --git a/regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.groovy b/regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.groovy new file mode 100644 index 00000000000000..ed3019436aef29 --- /dev/null +++ b/regression-test/suites/export_p0/outfile/test_outfile_jsonb_and_variant.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("test_outfile_jsonb_and_variant", "p0") { + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + def export_table_name = "test_outfile_jsonb_and_variant_table" + def outFilePath = "${bucket}/outfile/jsonb_and_variant/exp_" + + def outfile_to_S3 = { format -> + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${export_table_name} t + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS ${format} + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + + return res[0][3] + } + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE `${export_table_name}` ( + `dt` int(11) NULL COMMENT "", + `id` int(11) NULL COMMENT "", + `json_col` JSON NULL COMMENT "", + `variant_col` variant NULL COMMENT "" + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`dt`) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + INSERT INTO `${export_table_name}` values + (20220201,0, '{"k1": "100"}', '{"k1": "100"}'), + (20220201,1, '{"k1": "100", "k2": "123"}', '{"k1": "100", "k2": "123"}'), + (20220201,2, '{"k1": "100", "abc": "567"}', '{"k1": "100", "abc": "567"}'), + (20220201,3, '{"k1": "100", "k3": 123}', '{"k1": "100", "k3": 123}'), + (20220201,4, '{"k1": "100", "doris": "nereids"}', '{"k1": "100", "doris": "nereids"}'), + (20220201,5, '{"k1": "100", "doris": "pipeline"}', '{"k1": "100", "doris": "pipeline"}'); + """ + + // parquet file format + def format = "parquet" + def outfile_url = outfile_to_S3("${format}") + qt_select_load_parquet """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // orc file foramt + format = "orc" + outfile_url = outfile_to_S3("${format}") + qt_select_load_orc """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // orc file foramt + format = "csv" + outfile_url = outfile_to_S3("${format}") + qt_select_load_orc """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ +} \ No newline at end of file diff --git a/regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy b/regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy new file mode 100644 index 00000000000000..492b6770195341 --- /dev/null +++ b/regression-test/suites/export_p0/outfile/test_outfile_null_type.groovy @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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_outfile_null_type", "p0") { + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + def export_table_name = "test_outfile_null_type" + def outFilePath = "${bucket}/outfile/null_type/exp_" + + def outfile_to_S3 = { format -> + // select ... into outfile ... + def res = sql """ + SELECT *, NULL AS null_col FROM ${export_table_name} t + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS ${format} + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + + return res[0][3] + } + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE `${export_table_name}` ( + `id` int(11) NULL, + `Name` string NULL, + `age` int(11) NULL + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`id`) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + StringBuilder sb = new StringBuilder() + int i = 1 + for (; i < 10; i ++) { + sb.append(""" + (${i}, 'ftw-${i}', ${i + 18}), + """) + } + sb.append(""" + (${i}, NULL, NULL) + """) + sql """ INSERT INTO ${export_table_name} VALUES + ${sb.toString()} + """ + def insert_res = sql "show last insert;" + logger.info("insert result: " + insert_res.toString()) + qt_select_export """ SELECT * FROM ${export_table_name} t ORDER BY id; """ + + // parquet file format + def format = "parquet" + def outfile_url = outfile_to_S3("${format}") + order_qt_select_load_parquet """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ + + // TODO(ftw): orc supports to export data of NULL type. + // orc file foramt + // format = "orc" + // outfile_url = outfile_to_S3("${format}") + // qt_select_load_orc """ SELECT * FROM S3 ( + // "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + // "ACCESS_KEY"= "${ak}", + // "SECRET_KEY" = "${sk}", + // "format" = "${format}", + // "region" = "${region}" + // ); + // """ + + // csv file foramt + format = "csv" + outfile_url = outfile_to_S3("${format}") + order_qt_select_load_csv """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.${format}", + "ACCESS_KEY"= "${ak}", + "SECRET_KEY" = "${sk}", + "format" = "${format}", + "region" = "${region}" + ); + """ +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/hive/hive_json_basic_test.groovy b/regression-test/suites/external_table_p0/hive/hive_json_basic_test.groovy new file mode 100644 index 00000000000000..9d05e1a4c7403d --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/hive_json_basic_test.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("hive_json_basic_test", "p0,external,hive,external_docker,external_docker_hive") { + + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + + for (String hivePrefix : ["hive2", "hive3"]) { + try { + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String catalog_name = "${hivePrefix}_hive_json_basic_test" + String broker_name = "hdfs" + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + 'type'='hms', + 'hive.metastore.uris'='thrift://${externalEnvIp}:${hms_port}' + );""" + sql """use `${catalog_name}`.`default`""" + + String tb1 = """json_all_complex_types""" + String tb2 = """json_nested_complex_table""" + String tb3 = """json_load_data_table""" + + def tables = sql """ show tables """ + logger.info("tables = ${tables}") + + qt_q1 """ select * from ${tb1} order by id """ + qt_q2 """ select * from ${tb1} where tinyint_col < 0 order by id """ + qt_q3 """ select * from ${tb1} where bigint_col > 0 order by id """ + qt_q4 """ select float_col from ${tb1} where float_col is not null order by id """ + qt_q5 """ select * from ${tb1} where id = 2 order by id """ + + + + qt_q6 """ select * from ${tb2} order by user_id""" + qt_q7 """ select user_id,activity_log from ${tb2} order by user_id""" + + + order_qt_q8 """ select * from ${tb3} order by id """ + + order_qt_q9 """ select col1,id from ${tb3} order by id """ + + + + + sql """drop catalog if exists ${catalog_name}""" + } finally { + } + } +} diff --git a/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy b/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy new file mode 100644 index 00000000000000..48b8b6559ca82e --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/iceberg_read_unitycatalog_table.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("iceberg_read_unitycatalog_table", "p0,external,doris,external_docker,external_docker_doris") { + + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "iceberg_read_unitycatalog_table" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + logger.info("catalog " + catalog_name + " created") + sql """ use ${catalog_name}.test_db """ + String tb = "unitycatalog_marksheet_uniform" + + qt_q1 """ select * from ${tb} order by c1 """ + qt_q2 """ select c1 from ${tb} where c1 > 6 order by c1 """ + qt_q3 """ select c2, c3 from ${tb} where c3 > 200 order by c1 """ + +} + +/* + +spark-sql: + 1. create table marksheet_uniform (c1 int, c2 string, c3 int); + 2. get parquet file from marksheet_uniform; (ref: https://docs.unitycatalog.io/usage/tables/uniform/) + 3. put parquet file to hdfs: hdfs dfs -put hdfs://xxxxx + 4. CALL .system.add_files( + table => '.unitycatalog_db.marksheet_uniform', + source_table => '`parquet`.`hdfs://172.20.32.136:8020/user/doris/preinstalled_data/iceberg_hadoop_warehouse/unitycatalog_db/marksheet_uniform_data/part-00000-5af50cc4-3218-465b-a3a4-eb4fc709421d-c000.snappy.parquet`' + ); +*/ \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy index fba50c3341e8a8..7875ac4244e372 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_clickhouse_jdbc_catalog.groovy @@ -83,17 +83,11 @@ suite("test_clickhouse_jdbc_catalog", "p0,external,clickhouse,external_docker,ex sql("select * from ts where from_unixtime(ts,'yyyyMMdd') >= '2022-01-01';") contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE ((FROM_UNIXTIME("ts", '%Y%m%d') >= '2022-01-01'))""" } - explain { - sql("select * from ts where nvl(ts,null) >= '1';") - contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts""" - } order_qt_func_push2 """select * from ts where ts <= unix_timestamp(from_unixtime(ts,'yyyyMMdd'));""" - sql "set enable_jdbc_cast_predicate_push_down = true;" explain { sql("select * from ts where ts <= unix_timestamp(from_unixtime(ts,'yyyy-MM-dd'));") contains """QUERY: SELECT "id", "ts" FROM "doris_test"."ts" WHERE (("ts" <= toUnixTimestamp(FROM_UNIXTIME("ts", '%Y-%m-%d'))))""" } - sql "set enable_jdbc_cast_predicate_push_down = false;" order_qt_dt_with_tz """ select * from dt_with_tz order by id; """ diff --git a/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy b/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy index 6271a1619a5fac..3585f5fbb50941 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_jdbc_catalog_push_cast.groovy @@ -34,6 +34,8 @@ suite("test_jdbc_catalog_push_cast", "p0,external,mysql,external_docker,external "driver_class" = "com.mysql.cj.jdbc.Driver" );""" + sql "set enable_jdbc_cast_predicate_push_down = false;" + sql "use jdbc_catalog_push_cast.doris_test" qt_sql """select * from test_cast where date(datetime_c) = '2022-01-01';""" diff --git a/regression-test/suites/external_table_p0/jdbc/test_jdbc_row_count.groovy b/regression-test/suites/external_table_p0/jdbc/test_jdbc_row_count.groovy index 5cb1f6267e32cb..c8220ce13dd0b1 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_jdbc_row_count.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_jdbc_row_count.groovy @@ -102,6 +102,36 @@ suite("test_jdbc_row_count", "p0,external,mysql,external_docker,external_docker_ } assertEquals("3", result[0][2]) sql """drop catalog ${catalog_name}""" + + // Test oracle + catalog_name = "test_oracle_jdbc_row_count"; + String oracle_port = context.config.otherConfigs.get("oracle_11_port"); + String SID = "XE"; + driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/ojdbc8.jar" + sql """drop catalog if exists ${catalog_name} """ + sql """ + create catalog if not exists ${catalog_name} properties( + "type"="jdbc", + "user"="doris_test", + "password"="123456", + "jdbc_url" = "jdbc:oracle:thin:@${externalEnvIp}:${oracle_port}:${SID}", + "driver_url" = "${driver_url}", + "driver_class" = "oracle.jdbc.driver.OracleDriver" + ); + """ + sql """use ${catalog_name}.DORIS_TEST""" + result = sql """show table stats STUDENT""" + Thread.sleep(1000) + for (int i = 0; i < 30; i++) { + result = sql """show table stats STUDENT"""; + if (result[0][2] != "-1") { + break; + } + logger.info("Table row count not ready yet. Wait 1 second.") + Thread.sleep(1000) + } + assertTrue("4".equals(result[0][2]) || "-1".equals(result[0][2])) + sql """drop catalog ${catalog_name}""" } } diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy index f2082ef89c7a50..149eecf5817bd4 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_catalog.groovy @@ -36,4 +36,4 @@ suite("test_hudi_catalog", "p2,external,hudi,external_remote,external_remote_hud def tables = sql """ show tables; """ assertTrue(tables.size() > 0) sql """drop catalog if exists ${catalog_name};""" -} \ No newline at end of file +} diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy index 8cc1d2a852b8c4..885903646cc5b5 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_incremental.groovy @@ -60,7 +60,6 @@ suite("test_hudi_incremental", "p2,external,hudi,external_remote,external_remote "20241114152009764", "20241114152011901", ] - test_hudi_incremental_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_partition order by _hoodie_commit_time;" def timestamps_cow_partition = [ @@ -75,7 +74,6 @@ suite("test_hudi_incremental", "p2,external,hudi,external_remote,external_remote "20241114152147114", "20241114152156417", ] - test_hudi_incremental_querys("user_activity_log_cow_partition", timestamps_cow_partition) // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_non_partition order by _hoodie_commit_time;" def timestamps_mor_non_partition = [ @@ -90,7 +88,6 @@ suite("test_hudi_incremental", "p2,external,hudi,external_remote,external_remote "20241114152028770", "20241114152030746", ] - test_hudi_incremental_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_partition order by _hoodie_commit_time;" def timestamps_mor_partition = [ @@ -105,7 +102,18 @@ suite("test_hudi_incremental", "p2,external,hudi,external_remote,external_remote "20241114152323587", "20241114152334111", ] + + test_hudi_incremental_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) + test_hudi_incremental_querys("user_activity_log_cow_partition", timestamps_cow_partition) + test_hudi_incremental_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) + test_hudi_incremental_querys("user_activity_log_mor_partition", timestamps_mor_partition) + sql """set force_jni_scanner=true;""" + // don't support incremental query for cow table by jni reader + // test_hudi_incremental_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) + // test_hudi_incremental_querys("user_activity_log_cow_partition", timestamps_cow_partition) + test_hudi_incremental_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) test_hudi_incremental_querys("user_activity_log_mor_partition", timestamps_mor_partition) + // sql """set force_jni_scanner=false;""" sql """drop catalog if exists ${catalog_name};""" -} \ No newline at end of file +} diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_orc_tables.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_orc_tables.groovy new file mode 100644 index 00000000000000..43638a23881e0e --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_orc_tables.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. + +suite("test_hudi_orc_tables", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_orc_tables" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + qt_cow """ select * from orc_hudi_table_cow; """ + qt_mor """ select * from orc_hudi_table_mor; """ + + sql """drop catalog if exists ${catalog_name};""" +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_partition_prune.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_partition_prune.groovy new file mode 100644 index 00000000000000..eea17b01711b44 --- /dev/null +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_partition_prune.groovy @@ -0,0 +1,333 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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_hudi_partition_prune", "p2,external,hudi,external_remote,external_remote_hudi") { + String enabled = context.config.otherConfigs.get("enableExternalHudiTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable hudi test") + } + + String catalog_name = "test_hudi_partition_prune" + String props = context.config.otherConfigs.get("hudiEmrCatalog") + sql """drop catalog if exists ${catalog_name};""" + + for (String use_hive_sync_partition : ['true','false']) { + + sql """ + create catalog if not exists ${catalog_name} properties ( + ${props} + ,"use_hive_sync_partition"="${use_hive_sync_partition}" + ); + """ + + sql """ switch ${catalog_name};""" + sql """ use regression_hudi;""" + sql """ set enable_fallback_to_original_planner=false """ + + + + def one_partition_1_1 = """SELECT id,name,part1 FROM one_partition_tb WHERE part1 = 2024 ORDER BY id;""" + def one_partition_2_1 = """SELECT id,name,part1 FROM one_partition_tb WHERE part1 = 2025 ORDER BY id;""" + def one_partition_3_all = """SELECT id,name,part1 FROM one_partition_tb ORDER BY id;""" + def one_partition_4_all = """SELECT id,name,part1 FROM one_partition_tb WHERE id = 5 ORDER BY id;""" + def one_partition_5_1 = """SELECT id,name,part1 FROM one_partition_tb WHERE part1 = 2024 AND id >= 3 ORDER BY id;""" + + def two_partition_1_1 = """SELECT id,name,part1,part2 FROM two_partition_tb WHERE part1 = 'US' AND part2 = 1 ORDER BY id;""" + def two_partition_2_1 = """SELECT id,name,part1,part2 FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + def two_partition_3_2 = """SELECT id,name,part1,part2 FROM two_partition_tb WHERE part1 = 'US' ORDER BY id;""" + def two_partition_4_all = """SELECT id,name,part1,part2 FROM two_partition_tb ORDER BY id;""" + def two_partition_5_1 = """SELECT id,name,part1,part2 FROM two_partition_tb WHERE part1 = 'US' AND part2 = 2 AND id > 5 ORDER BY id;""" + def two_partition_6_1 = """SELECT id,name,part1,part2 FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + + def three_partition_1_1 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_2_1 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2025 AND part3 = 'Q2' ORDER BY id;""" + def three_partition_3_3 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 ORDER BY id;""" + def three_partition_4_2 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q1' ORDER BY id;""" + def three_partition_5_all = """SELECT id,name,part1,part2,part3 FROM three_partition_tb ORDER BY id;""" + def three_partition_6_1 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_7_7 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part2 = 2025 ORDER BY id;""" + def three_partition_8_2 = """SELECT id,name,part1,part2,part3 FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q2' AND id BETWEEN 6 AND 10 ORDER BY id;""" + + def one_partition_boolean = """SELECT id,name,part1 FROM boolean_partition_tb WHERE part1 = true ORDER BY id;""" + def one_partition_tinyint = """SELECT id,name,part1 FROM tinyint_partition_tb WHERE part1 = 1 ORDER BY id;""" + def one_partition_smallint = """SELECT id,name,part1 FROM smallint_partition_tb WHERE part1 = 10 ORDER BY id;""" + def one_partition_int = """SELECT id,name,part1 FROM int_partition_tb WHERE part1 = 100 ORDER BY id;""" + def one_partition_bigint = """SELECT id,name,part1 FROM bigint_partition_tb WHERE part1 = 1234567890 ORDER BY id;""" + def one_partition_string = """SELECT id,name,part1 FROM string_partition_tb WHERE part1 = 'RegionA' ORDER BY id;""" + def one_partition_date = """SELECT id,name,part1 FROM date_partition_tb WHERE part1 = '2023-12-01' ORDER BY id;""" + def one_partition_timestamp = """SELECT id,name,part1 FROM timestamp_partition_tb WHERE part1 = '2023-12-01 08:00:00' ORDER BY id;""" + + + + qt_one_partition_1_1 one_partition_1_1 + explain { + sql("${one_partition_1_1}") + contains "partition=1/2" + } + + qt_one_partition_2_1 one_partition_2_1 + explain { + sql("${one_partition_2_1}") + contains "partition=1/2" + } + + qt_one_partition_3_all one_partition_3_all + explain { + sql("${one_partition_3_all}") + contains "partition=2/2" + } + + qt_one_partition_4_all one_partition_4_all + explain { + sql("${one_partition_4_all}") + contains "partition=2/2" + } + + qt_one_partition_5_1 one_partition_5_1 + explain { + sql("${one_partition_5_1}") + contains "partition=1/2" + } + + + qt_two_partition_1_1 two_partition_1_1 + explain { + sql("${two_partition_1_1}") + contains "partition=1/4" + } + + qt_two_partition_2_1 two_partition_2_1 + explain { + sql("${two_partition_2_1}") + contains "partition=1/4" + } + + qt_two_partition_3_2 two_partition_3_2 + explain { + sql("${two_partition_3_2}") + contains "partition=2/4" + } + + qt_two_partition_4_all two_partition_4_all + explain { + sql("${two_partition_4_all}") + contains "partition=4/4" + } + + qt_two_partition_5_1 two_partition_5_1 + explain { + sql("${two_partition_5_1}") + contains "partition=1/4" + } + + qt_two_partition_6_1 two_partition_6_1 + explain { + sql("${two_partition_6_1}") + contains "partition=1/4" + } + + + + qt_three_partition_1_1 three_partition_1_1 + explain { + sql("${three_partition_1_1}") + contains "partition=1/10" + } + + qt_three_partition_2_1 three_partition_2_1 + explain { + sql("${three_partition_2_1}") + contains "partition=1/10" + } + + qt_three_partition_3_3 three_partition_3_3 + explain { + sql("${three_partition_3_3}") + contains "partition=3/10" + } + + qt_three_partition_4_2 three_partition_4_2 + explain { + sql("${three_partition_4_2}") + contains "partition=2/10" + } + + qt_three_partition_5_all three_partition_5_all + explain { + sql("${three_partition_5_all}") + contains "partition=10/10" + } + + qt_three_partition_6_1 three_partition_6_1 + explain { + sql("${three_partition_6_1}") + contains "partition=1/10" + } + + qt_three_partition_7_7 three_partition_7_7 + explain { + sql("${three_partition_7_7}") + contains "partition=7/10" + } + + qt_three_partition_8_2 three_partition_8_2 + explain { + sql("${three_partition_8_2}") + contains "partition=2/10" + } + + + // 0 partitions + def one_partition_6_0 = """SELECT id,name,part1 FROM one_partition_tb WHERE part1 = 2023 ORDER BY id;""" + qt_one_partition_6_0 one_partition_6_0 + explain { + sql("${one_partition_6_0}") + contains "partition=0/2" + } + + def two_partition_7_0 = """SELECT id,name,part1 FROM two_partition_tb WHERE part1 = 'CN' AND part2 = 1 ORDER BY id;""" + qt_two_partition_7_0 two_partition_7_0 + explain { + sql("${two_partition_7_0}") + contains "partition=0/4" + } + + def two_partition_8_0 = """SELECT id,name,part1 FROM two_partition_tb WHERE part1 = 'US' AND part2 = 3 ORDER BY id;""" + qt_two_partition_8_0 two_partition_8_0 + explain { + sql("${two_partition_8_0}") + contains "partition=0/4" + } + + def three_partition_9_0 = """SELECT id,name,part1 FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2023 AND part3 = 'Q1' ORDER BY id;""" + qt_three_partition_9_0 three_partition_9_0 + explain { + sql("${three_partition_9_0}") + contains "partition=0/10" + } + + def three_partition_10_0 = """SELECT id,name,part1 FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_10_0 three_partition_10_0 + explain { + sql("${three_partition_10_0}") + contains "partition=0/10" + } + + def three_partition_11_0 = """SELECT id,name,part1 FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_11_0 three_partition_11_0 + explain { + sql("${three_partition_11_0}") + contains "partition=0/10" + } + + + //time travel + def time_travel_two_partition_1_3 = "select id,name,part1,part2 from two_partition_tb FOR TIME AS OF '20241202171226401' order by id;" + def time_travel_two_partition_2_2 = "select id,name,part1,part2 from two_partition_tb FOR TIME AS OF '20241202171226401' where part1='US' order by id;" + def time_travel_two_partition_3_1 = "select id,name,part1,part2 from two_partition_tb FOR TIME AS OF '20241202171226401' where part2=2 order by id;" + def time_travel_two_partition_4_0 = "select id,name,part1,part2 from two_partition_tb FOR TIME AS OF '20241202171226401' where part2=10 order by id;" + + qt_time_travel_two_partition_1_3 time_travel_two_partition_1_3 + explain { + sql("${time_travel_two_partition_1_3}") + contains "partition=3/3" + } + + + qt_time_travel_two_partition_2_2 time_travel_two_partition_2_2 + explain { + sql("${time_travel_two_partition_2_2}") + contains "partition=2/3" + } + + qt_time_travel_two_partition_3_1 time_travel_two_partition_3_1 + explain { + sql("${time_travel_two_partition_3_1}") + contains "partition=1/3" + } + + qt_time_travel_two_partition_4_0 time_travel_two_partition_4_0 + explain { + sql("${time_travel_two_partition_4_0}") + contains "partition=0/3" + } + + + + + def time_travel_two_partition_5_0 = "select id,name,part1,part2 from two_partition_tb FOR TIME AS OF '20231126012025218' order by id;" + qt_time_travel_two_partition_5_0 time_travel_two_partition_5_0 + explain { + sql("${time_travel_two_partition_5_0}") + contains "partition=0/0" + } + + def time_travel_two_partition_6_1 = "select id,name,part1,part2 from two_partition_tb FOR TIME AS OF '20241202171214902' order by id;" + qt_time_travel_two_partition_6_1 time_travel_two_partition_6_1 + explain { + sql("${time_travel_two_partition_6_1}") + contains "partition=1/1" + } + + // all types as partition + qt_one_partition_boolean one_partition_boolean + explain { + sql("${one_partition_boolean}") + contains "partition=1/2" + } + qt_one_partition_tinyint one_partition_tinyint + explain { + sql("${one_partition_tinyint}") + contains "partition=1/2" + } + qt_one_partition_smallint one_partition_smallint + explain { + sql("${one_partition_smallint}") + contains "partition=1/2" + } + qt_one_partition_int one_partition_int + explain { + sql("${one_partition_int}") + contains "partition=1/2" + } + qt_one_partition_bigint one_partition_bigint + explain { + sql("${one_partition_bigint}") + contains "partition=1/2" + } + qt_one_partition_string one_partition_string + explain { + sql("${one_partition_string}") + contains "partition=1/2" + } + qt_one_partition_date one_partition_date + explain { + sql("${one_partition_date}") + contains "partition=1/2" + } + // qt_one_partition_timestamp one_partition_timestamp + // explain { + // sql("${one_partition_timestamp}") + // contains "partition=1/2" + // } + + sql """drop catalog if exists ${catalog_name};""" + + + } + +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy index b247aaf492400d..0da88447cdef15 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_schema_evolution.groovy @@ -33,7 +33,18 @@ suite("test_hudi_schema_evolution", "p2,external,hudi,external_remote,external_r sql """ switch ${catalog_name};""" sql """ use regression_hudi;""" sql """ set enable_fallback_to_original_planner=false """ + + qt_adding_simple_columns_table """ select * from adding_simple_columns_table order by id """ + qt_altering_simple_columns_table """ select * from altering_simple_columns_table order by id """ + // qt_deleting_simple_columns_table """ select * from deleting_simple_columns_table order by id """ + // qt_renaming_simple_columns_table """ select * from renaming_simple_columns_table order by id """ + qt_adding_complex_columns_table """ select * from adding_complex_columns_table order by id """ + qt_altering_complex_columns_table """ select * from altering_complex_columns_table order by id """ + // qt_deleting_complex_columns_table """ select * from deleting_complex_columns_table order by id """ + // qt_renaming_complex_columns_table """ select * from renaming_complex_columns_table order by id """ + + sql """set force_jni_scanner = true;""" qt_adding_simple_columns_table """ select * from adding_simple_columns_table order by id """ qt_altering_simple_columns_table """ select * from altering_simple_columns_table order by id """ // qt_deleting_simple_columns_table """ select * from deleting_simple_columns_table order by id """ @@ -43,6 +54,7 @@ suite("test_hudi_schema_evolution", "p2,external,hudi,external_remote,external_r qt_altering_complex_columns_table """ select * from altering_complex_columns_table order by id """ // qt_deleting_complex_columns_table """ select * from deleting_complex_columns_table order by id """ // qt_renaming_complex_columns_table """ select * from renaming_complex_columns_table order by id """ + sql """set force_jni_scanner = false;""" sql """drop catalog if exists ${catalog_name};""" -} \ No newline at end of file +} diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy index 53c09e6d5a9031..89d89709b3c822 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_snapshot.groovy @@ -64,7 +64,7 @@ suite("test_hudi_snapshot", "p2,external,hudi,external_remote,external_remote_hu qt_q09 """SELECT * FROM ${table_name} WHERE struct_element(struct_element(address, 'coordinates'), 'latitude') BETWEEN 0 AND 100 AND struct_element(struct_element(address, 'coordinates'), 'longitude') BETWEEN 0 AND 100 ORDER BY event_time LIMIT 5;""" // Query records with ratings above a specific value and limit output - qt_q10 """SELECT * FROM ${table_name} WHERE rating > 4.5 ORDER BY rating DESC LIMIT 5;""" + qt_q10 """SELECT * FROM ${table_name} WHERE rating > 4.5 ORDER BY event_time DESC LIMIT 5;""" // Query all users' signup dates and limit output qt_q11 """SELECT user_id, signup_date FROM ${table_name} ORDER BY signup_date DESC LIMIT 10;""" @@ -79,13 +79,20 @@ suite("test_hudi_snapshot", "p2,external,hudi,external_remote,external_remote_hu qt_q14 """SELECT * FROM ${table_name} WHERE signup_date = '2024-01-15' ORDER BY user_id LIMIT 5;""" // Query the total count of purchases for each user and limit output - qt_q15 """SELECT user_id, array_size(purchases) AS purchase_count FROM ${table_name} ORDER BY purchase_count DESC LIMIT 5;""" + qt_q15 """SELECT user_id, array_size(purchases) AS purchase_count FROM ${table_name} ORDER BY user_id LIMIT 5;""" } + test_hudi_snapshot_querys("user_activity_log_mor_non_partition") + test_hudi_snapshot_querys("user_activity_log_mor_partition") test_hudi_snapshot_querys("user_activity_log_cow_non_partition") test_hudi_snapshot_querys("user_activity_log_cow_partition") + + sql """set force_jni_scanner=true;""" test_hudi_snapshot_querys("user_activity_log_mor_non_partition") test_hudi_snapshot_querys("user_activity_log_mor_partition") + test_hudi_snapshot_querys("user_activity_log_cow_non_partition") + test_hudi_snapshot_querys("user_activity_log_cow_partition") + sql """set force_jni_scanner=false;""" sql """drop catalog if exists ${catalog_name};""" -} \ No newline at end of file +} diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy index c1ba630e4a7d01..3d7bd40b2d54cf 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_timestamp.groovy @@ -34,8 +34,22 @@ suite("test_hudi_timestamp", "p2,external,hudi,external_remote,external_remote_h sql """ use regression_hudi;""" sql """ set enable_fallback_to_original_planner=false """ - // TODO: fix hudi timezone issue and enable this - // qt_timestamp """ select * from hudi_table_with_timestamp order by id; """ + def test_timestamp_different_timezones = { + sql """set time_zone = 'America/Los_Angeles';""" + qt_timestamp1 """ select * from hudi_table_with_timestamp order by id; """ + sql """set time_zone = 'Asia/Shanghai';""" + qt_timestamp2 """ select * from hudi_table_with_timestamp order by id; """ + sql """set time_zone = 'UTC';""" + qt_timestamp3 """ select * from hudi_table_with_timestamp order by id; """ + } + + // test native reader + test_timestamp_different_timezones() + sql """ set force_jni_scanner = true; """ + // test jni reader + test_timestamp_different_timezones() + sql """ set force_jni_scanner = false; """ + sql """drop catalog if exists ${catalog_name};""" } @@ -59,4 +73,4 @@ suite("test_hudi_timestamp", "p2,external,hudi,external_remote,external_remote_h // INSERT OVERWRITE hudi_table_with_timestamp VALUES // ('1', 'Alice', timestamp('2024-10-25 08:00:00')), // ('2', 'Bob', timestamp('2024-10-25 09:30:00')), -// ('3', 'Charlie', timestamp('2024-10-25 11:00:00')); \ No newline at end of file +// ('3', 'Charlie', timestamp('2024-10-25 11:00:00')); diff --git a/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy b/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy index 4d458dc4381dcf..cceeaa412202c6 100644 --- a/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy +++ b/regression-test/suites/external_table_p2/hudi/test_hudi_timetravel.groovy @@ -54,7 +54,6 @@ suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_ "20241114152009764", "20241114152011901", ] - test_hudi_timetravel_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) // spark-sql "select distinct _hoodie_commit_time from user_activity_log_cow_partition order by _hoodie_commit_time;" def timestamps_cow_partition = [ @@ -69,7 +68,6 @@ suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_ "20241114152147114", "20241114152156417", ] - test_hudi_timetravel_querys("user_activity_log_cow_partition", timestamps_cow_partition) // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_non_partition order by _hoodie_commit_time;" def timestamps_mor_non_partition = [ @@ -84,7 +82,6 @@ suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_ "20241114152028770", "20241114152030746", ] - test_hudi_timetravel_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) // spark-sql "select distinct _hoodie_commit_time from user_activity_log_mor_partition order by _hoodie_commit_time;" def timestamps_mor_partition = [ @@ -99,7 +96,17 @@ suite("test_hudi_timetravel", "p2,external,hudi,external_remote,external_remote_ "20241114152323587", "20241114152334111", ] + + test_hudi_timetravel_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) + test_hudi_timetravel_querys("user_activity_log_cow_partition", timestamps_cow_partition) + test_hudi_timetravel_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) + test_hudi_timetravel_querys("user_activity_log_mor_partition", timestamps_mor_partition) + sql """set force_jni_scanner=true;""" + test_hudi_timetravel_querys("user_activity_log_cow_non_partition", timestamps_cow_non_partition) + test_hudi_timetravel_querys("user_activity_log_cow_partition", timestamps_cow_partition) + test_hudi_timetravel_querys("user_activity_log_mor_non_partition", timestamps_mor_non_partition) test_hudi_timetravel_querys("user_activity_log_mor_partition", timestamps_mor_partition) + sql """set force_jni_scanner=false;""" sql """drop catalog if exists ${catalog_name};""" -} \ No newline at end of file +} diff --git a/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy b/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy index 3f0929b59ea96a..81133270fb6e60 100644 --- a/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy +++ b/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy @@ -388,7 +388,10 @@ suite("test_external_catalog_maxcompute", "p2,external,maxcompute,external_remot order_qt_multi_partition_q6 """ select max(pt), yy, mm from multi_partitions where yy = '2023' and mm='08' group by yy, mm order by yy, mm; """ order_qt_multi_partition_q7 """ select count(*) from multi_partitions where yy < '2023' or dd < '03'; """ order_qt_multi_partition_q8 """ select count(*) from multi_partitions where pt>=3; """ - order_qt_multi_partition_q9 """ select city,mnt,gender,finished_time,order_rate,cut_date,create_time,pt, yy, mm, dd from multi_partitions where pt >= 2 and pt < 4 and finished_time is not null; """ + + //`finished_time is not null` => com.aliyun.odps.OdpsException: ODPS-0010000:System internal error - fuxi job failed, caused by: timestamp_ntz + // order_qt_multi_partition_q9 """ select city,mnt,gender,finished_time,order_rate,cut_date,create_time,pt, yy, mm, dd from multi_partitions where pt >= 2 and pt < 4 and finished_time is not null; """ + order_qt_multi_partition_q10 """ select pt, yy, mm, dd from multi_partitions where pt >= 2 and create_time > '2023-08-03 03:11:00' order by pt, yy, mm, dd; """ diff --git a/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy new file mode 100644 index 00000000000000..83ebbe2fb29d1a --- /dev/null +++ b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy @@ -0,0 +1,294 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + + +/* +CREATE TABLE one_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 INT); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (1, 'Alice'); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (2, 'Bob'); +INSERT INTO one_partition_tb PARTITION (part1=2024) VALUES (3, 'Charlie'); +INSERT INTO one_partition_tb PARTITION (part1=2025) VALUES (4, 'David'); +INSERT INTO one_partition_tb PARTITION (part1=2025) VALUES (5, 'Eva'); +CREATE TABLE two_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 STRING, part2 int); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (1, 'Alice'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (2, 'Bob'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=1) VALUES (3, 'Charlie'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=2) VALUES (4, 'David'); +INSERT INTO two_partition_tb PARTITION (part1='US', part2=2) VALUES (5, 'Eva'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=1) VALUES (6, 'Frank'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=1) VALUES (7, 'Grace'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (8, 'Hannah'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (9, 'Ivy'); +INSERT INTO two_partition_tb PARTITION (part1='EU', part2=2) VALUES (10, 'Jack'); +CREATE TABLE three_partition_tb ( + id INT, + name string +) +PARTITIONED BY (part1 STRING, part2 INT, part3 STRING); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (1, 'Alice'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (2, 'Bob'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q1') VALUES (3, 'Charlie'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q2') VALUES (4, 'David'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2024, part3='Q2') VALUES (5, 'Eva'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2025, part3='Q1') VALUES (6, 'Frank'); +INSERT INTO three_partition_tb PARTITION (part1='US', part2=2025, part3='Q2') VALUES (7, 'Grace'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2024, part3='Q1') VALUES (8, 'Hannah'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2024, part3='Q1') VALUES (9, 'Ivy'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q2') VALUES (10, 'Jack'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q2') VALUES (11, 'Leo'); +INSERT INTO three_partition_tb PARTITION (part1='EU', part2=2025, part3='Q3') VALUES (12, 'Mia'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q1') VALUES (13, 'Nina'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q2') VALUES (14, 'Oscar'); +INSERT INTO three_partition_tb PARTITION (part1='AS', part2=2025, part3='Q3') VALUES (15, 'Paul'); +select * from one_partition_tb; +select * from two_partition_tb; +select * from three_partition_tb; +show partitions one_partition_tb; +show partitions two_partition_tb; +show partitions three_partition_tb; +*/ + +suite("test_max_compute_partition_prune", "p2,external,maxcompute,external_remote,external_remote_maxcompute") { + + + def one_partition_1_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2024 ORDER BY id;""" + def one_partition_2_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2025 ORDER BY id;""" + def one_partition_3_all = """SELECT * FROM one_partition_tb ORDER BY id;""" + def one_partition_4_all = """SELECT * FROM one_partition_tb WHERE id = 5 ORDER BY id;""" + def one_partition_5_1 = """SELECT * FROM one_partition_tb WHERE part1 = 2024 AND id >= 3 ORDER BY id;""" + + def two_partition_1_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 1 ORDER BY id;""" + def two_partition_2_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + def two_partition_3_2 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' ORDER BY id;""" + def two_partition_4_all = """SELECT * FROM two_partition_tb ORDER BY id;""" + def two_partition_5_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 2 AND id > 5 ORDER BY id;""" + def two_partition_6_1 = """SELECT * FROM two_partition_tb WHERE part1 = 'EU' AND part2 = 2 ORDER BY id;""" + + def three_partition_1_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_2_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2025 AND part3 = 'Q2' ORDER BY id;""" + def three_partition_3_3 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 ORDER BY id;""" + def three_partition_4_2 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q1' ORDER BY id;""" + def three_partition_5_all = """SELECT * FROM three_partition_tb ORDER BY id;""" + def three_partition_6_1 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q1' ORDER BY id;""" + def three_partition_7_7 = """SELECT * FROM three_partition_tb WHERE part2 = 2025 ORDER BY id;""" + def three_partition_8_2 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part3 = 'Q2' AND id BETWEEN 6 AND 10 ORDER BY id;""" + + + String enabled = context.config.otherConfigs.get("enableMaxComputeTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String ak = context.config.otherConfigs.get("ak") + String sk = context.config.otherConfigs.get("sk"); + String mc_db = "mc_datalake" + String mc_catalog_name = "test_max_compute_partition_prune" + + + for (String enable_profile : ["true","false"] ) { + sql """set enable_profile = ${enable_profile} """; + + for (String num_partitions : ["1","10","100"] ) { + sql "set num_partitions_in_batch_mode = ${num_partitions} " + + for (String cross_partition : ["true","false"] ) { + + sql """drop catalog if exists ${mc_catalog_name};""" + sql """ + create catalog if not exists ${mc_catalog_name} properties ( + "type" = "max_compute", + "mc.default.project" = "${mc_db}", + "mc.access_key" = "${ak}", + "mc.secret_key" = "${sk}", + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api", + "mc.split_cross_partition" = "${cross_partition}" + ); + """ + sql """ switch ${mc_catalog_name} """ + sql """ use ${mc_db}""" + + qt_one_partition_1_1 one_partition_1_1 + explain { + sql("${one_partition_1_1}") + contains "partition=1/2" + } + + qt_one_partition_2_1 one_partition_2_1 + explain { + sql("${one_partition_2_1}") + contains "partition=1/2" + } + + qt_one_partition_3_all one_partition_3_all + explain { + sql("${one_partition_3_all}") + contains "partition=2/2" + } + + qt_one_partition_4_all one_partition_4_all + explain { + sql("${one_partition_4_all}") + contains "partition=2/2" + } + + qt_one_partition_5_1 one_partition_5_1 + explain { + sql("${one_partition_5_1}") + contains "partition=1/2" + } + + + qt_two_partition_1_1 two_partition_1_1 + explain { + sql("${two_partition_1_1}") + contains "partition=1/4" + } + + qt_two_partition_2_1 two_partition_2_1 + explain { + sql("${two_partition_2_1}") + contains "partition=1/4" + } + + qt_two_partition_3_2 two_partition_3_2 + explain { + sql("${two_partition_3_2}") + contains "partition=2/4" + } + + qt_two_partition_4_all two_partition_4_all + explain { + sql("${two_partition_4_all}") + contains "partition=4/4" + } + + qt_two_partition_5_1 two_partition_5_1 + explain { + sql("${two_partition_5_1}") + contains "partition=1/4" + } + + qt_two_partition_6_1 two_partition_6_1 + explain { + sql("${two_partition_6_1}") + contains "partition=1/4" + } + + + + qt_three_partition_1_1 three_partition_1_1 + explain { + sql("${three_partition_1_1}") + contains "partition=1/10" + } + + qt_three_partition_2_1 three_partition_2_1 + explain { + sql("${three_partition_2_1}") + contains "partition=1/10" + } + + qt_three_partition_3_3 three_partition_3_3 + explain { + sql("${three_partition_3_3}") + contains "partition=3/10" + } + + qt_three_partition_4_2 three_partition_4_2 + explain { + sql("${three_partition_4_2}") + contains "partition=2/10" + } + + qt_three_partition_5_all three_partition_5_all + explain { + sql("${three_partition_5_all}") + contains "partition=10/10" + } + + qt_three_partition_6_1 three_partition_6_1 + explain { + sql("${three_partition_6_1}") + contains "partition=1/10" + } + + qt_three_partition_7_7 three_partition_7_7 + explain { + sql("${three_partition_7_7}") + contains "partition=7/10" + } + + qt_three_partition_8_2 three_partition_8_2 + explain { + sql("${three_partition_8_2}") + contains "partition=2/10" + } + + + // 0 partitions + def one_partition_6_0 = """SELECT * FROM one_partition_tb WHERE part1 = 2023 ORDER BY id;""" + qt_one_partition_6_0 one_partition_6_0 + explain { + sql("${one_partition_6_0}") + contains "partition=0/2" + } + + def two_partition_7_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'CN' AND part2 = 1 ORDER BY id;""" + qt_two_partition_7_0 two_partition_7_0 + explain { + sql("${two_partition_7_0}") + contains "partition=0/4" + } + + def two_partition_8_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 3 ORDER BY id;""" + qt_two_partition_8_0 two_partition_8_0 + explain { + sql("${two_partition_8_0}") + contains "partition=0/4" + } + + def three_partition_9_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2023 AND part3 = 'Q1' ORDER BY id;""" + qt_three_partition_9_0 three_partition_9_0 + explain { + sql("${three_partition_9_0}") + contains "partition=0/10" + } + + def three_partition_10_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_10_0 three_partition_10_0 + explain { + sql("${three_partition_10_0}") + contains "partition=0/10" + } + + def three_partition_11_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_11_0 three_partition_11_0 + explain { + sql("${three_partition_11_0}") + contains "partition=0/10" + } + } + } + } + } +} \ No newline at end of file diff --git a/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy index fc2abe7fb0ad6f..bccaa8aa62f84d 100644 --- a/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_load_stream_back_pressure_fault_injection.groovy @@ -71,67 +71,32 @@ suite("test_load_stream_back_pressure_fault_injection", "nonConcurrent") { try { GetDebugPoint().enableDebugPointForAllBEs("TabletStream.append_data.long_wait") - def thread1 = new Thread({ - try { - def res = sql "insert into test select * from baseall where k1 <= 3" - logger.info(res.toString()) - } catch(Exception e) { - logger.info(e.getMessage()) - assertTrue(e.getMessage().contains("Communications link failure")) + // the kill thread only means to end the test faster when the code does not behave as expected + def kill_thread = new Thread({ + sleep(5000) + def processList = sql "show processlist" + logger.info(processList.toString()) + processList.each { item -> + logger.info(item[1].toString()) + logger.info(item[11].toString()) + if (item[11].toString() == "insert into test select * from baseall where k1 <= 3".toString()){ + def res = sql "kill ${item[1]}" + logger.info(res.toString()) + } } }) - thread1.start() - - sleep(1000) - - def processList = sql "show processlist" - logger.info(processList.toString()) - processList.each { item -> - logger.info(item[1].toString()) - logger.info(item[11].toString()) - if (item[11].toString() == "insert into test select * from baseall where k1 <= 3".toString()){ - def res = sql "kill ${item[1]}" - logger.info(res.toString()) - } - } + kill_thread.start() + def res = sql "insert into test select * from baseall where k1 <= 3" + logger.info(res.toString()) + assertTrue(false, "Expected exception to be thrown") } catch(Exception e) { logger.info(e.getMessage()) + assertTrue(e.getMessage().contains("wait flush token back pressure time is more than load_stream_max_wait_flush_token_time")) } finally { GetDebugPoint().disableDebugPointForAllBEs("TabletStream.append_data.long_wait") } - try { - GetDebugPoint().enableDebugPointForAllBEs("TabletStream.add_segment.long_wait") - def thread1 = new Thread({ - try { - def res = sql "insert into test select * from baseall where k1 <= 3" - logger.info(res.toString()) - } catch(Exception e) { - logger.info(e.getMessage()) - assertTrue(e.getMessage().contains("Communications link failure")) - } - }) - thread1.start() - - sleep(1000) - - def processList = sql "show processlist" - logger.info(processList.toString()) - processList.each { item -> - logger.info(item[1].toString()) - logger.info(item[11].toString()) - if (item[11].toString() == "insert into test select * from baseall where k1 <= 3".toString()){ - def res = sql "kill ${item[1]}" - logger.info(res.toString()) - } - } - } catch(Exception e) { - logger.info(e.getMessage()) - } finally { - GetDebugPoint().disableDebugPointForAllBEs("TabletStream.add_segment.long_wait") - } - sql """ DROP TABLE IF EXISTS `baseall` """ sql """ DROP TABLE IF EXISTS `test` """ sql """ set enable_memtable_on_sink_node=false """ -} \ No newline at end of file +} diff --git a/regression-test/suites/fault_injection_p0/test_load_stream_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_load_stream_fault_injection.groovy index d38141ed55aafe..0ba05394adf657 100644 --- a/regression-test/suites/fault_injection_p0/test_load_stream_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_load_stream_fault_injection.groovy @@ -113,10 +113,11 @@ suite("load_stream_fault_injection", "nonConcurrent") { } } - def load_with_injection = { injection, expect_errmsg -> + def load_with_injection = { injection, expect_errmsg, success=false-> try { GetDebugPoint().enableDebugPointForAllBEs(injection) sql "insert into test select * from baseall where k1 <= 3" + assertTrue(success, String.format("Expected Exception '%s', actual success", expect_errmsg)) } catch(Exception e) { // assertTrue(e.getMessage().contains("Process has no memory available")) // the msg should contain the root cause logger.info(e.getMessage()) @@ -125,11 +126,12 @@ suite("load_stream_fault_injection", "nonConcurrent") { } } - def load_with_injection2 = { injection1, injection2, error_msg-> + def load_with_injection2 = { injection1, injection2, error_msg, success=false-> try { GetDebugPoint().enableDebugPointForAllBEs(injection1) GetDebugPoint().enableDebugPointForAllBEs(injection2) sql "insert into test select * from baseall where k1 <= 3" + assertTrue(success, String.format("expected Exception '%s', actual success", expect_errmsg)) } catch(Exception e) { logger.info(e.getMessage()) assertTrue(e.getMessage().contains(error_msg)) @@ -155,8 +157,6 @@ suite("load_stream_fault_injection", "nonConcurrent") { load_with_injection("LoadStreamWriter.close.inverted_writers_size_not_match", "") load_with_injection("LoadStreamWriter.close.file_not_closed", "") load_with_injection("LoadStreamWriter.close.inverted_file_not_closed", "") - // LoadStreamWriter close_writer meet bytes_appended and real file size not match error - load_with_injection("FileWriter.close_writer.zero_bytes_appended", "") // LoadStreamWriter close_writer/add_segment meet not inited error load_with_injection("TabletStream.init.uninited_writer", "") // LoadStreamWriter init failure @@ -168,8 +168,6 @@ suite("load_stream_fault_injection", "nonConcurrent") { // LoadStreamWriter add_segment meet null file writer error load_with_injection("LoadStreamWriter._calc_file_size.null_file_writer", "") load_with_injection("LoadStreamWriter._calc_file_size.file_not_closed", "") - // LoadStreamWriter add_segment meet bytes_appended and real file size not match error - load_with_injection("FileWriter.add_segment.zero_bytes_appended", "") // LoadStream init failed coz LoadStreamWriter init failed load_with_injection("RowsetBuilder.check_tablet_version_count.too_many_version", "") // LoadStream add_segment meet unknown segid in request header diff --git a/regression-test/suites/fault_injection_p0/test_load_stream_stub_close_wait_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_load_stream_stub_close_wait_fault_injection.groovy index 4a87f1daf6b2d9..58b6ba4a075e33 100644 --- a/regression-test/suites/fault_injection_p0/test_load_stream_stub_close_wait_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_load_stream_stub_close_wait_fault_injection.groovy @@ -74,6 +74,7 @@ suite("test_load_stream_stub_close_wait_fault_injection", "nonConcurrent") { GetDebugPoint().enableDebugPointForAllBEs("LoadStreamStub::close_wait.long_wait") def res = sql "insert into test select * from baseall where k1 <= 3" logger.info(res.toString()) + assertTrue(false, "Expected Exception to be thrown") } catch(Exception e) { logger.info(e.getMessage()) assertTrue(e.getMessage().contains("cancel")) @@ -85,4 +86,4 @@ suite("test_load_stream_stub_close_wait_fault_injection", "nonConcurrent") { sql """ DROP TABLE IF EXISTS `baseall` """ sql """ DROP TABLE IF EXISTS `test` """ sql """ set enable_memtable_on_sink_node=false """ -} \ No newline at end of file +} diff --git a/regression-test/suites/fault_injection_p0/test_load_stream_stub_failure_injection.groovy b/regression-test/suites/fault_injection_p0/test_load_stream_stub_failure_injection.groovy index 5b1f9fba05b4bc..48c32883302e40 100644 --- a/regression-test/suites/fault_injection_p0/test_load_stream_stub_failure_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_load_stream_stub_failure_injection.groovy @@ -69,10 +69,11 @@ suite("test_stream_stub_fault_injection", "nonConcurrent") { file "baseall.txt" } - def load_with_injection = { injection, error_msg-> + def load_with_injection = { injection, error_msg, success=false-> try { GetDebugPoint().enableDebugPointForAllBEs(injection) sql "insert into test select * from baseall where k1 <= 3" + assertTrue(success, String.format("Expected Exception '%s', actual success", expect_errmsg)) } catch(Exception e) { logger.info(e.getMessage()) assertTrue(e.getMessage().contains(error_msg)) @@ -87,8 +88,6 @@ suite("test_stream_stub_fault_injection", "nonConcurrent") { load_with_injection("StreamSinkFileWriter.finalize.finalize_failed", "failed to send segment eos to any replicas") // LoadStreams stream wait failed load_with_injection("LoadStreamStub._send_with_retry.stream_write_failed", "StreamWrite failed, err=32") - // LoadStreams keeping stream when release - load_with_injection("LoadStreams.release.keeping_streams", "") sql """ DROP TABLE IF EXISTS `baseall` """ sql """ DROP TABLE IF EXISTS `test` """ diff --git a/regression-test/suites/fault_injection_p0/test_memtable_flush_fault.groovy b/regression-test/suites/fault_injection_p0/test_memtable_flush_fault.groovy index 5b954f1171ce7c..0ba2f0863e2e6b 100644 --- a/regression-test/suites/fault_injection_p0/test_memtable_flush_fault.groovy +++ b/regression-test/suites/fault_injection_p0/test_memtable_flush_fault.groovy @@ -55,6 +55,7 @@ suite("test_memtable_flush_fault", "nonConcurrent") { GetDebugPoint().enableDebugPointForAllBEs("FlushToken.submit_flush_error") sql insert_sql sql "sync" + assertTrue(false, "Expected Exception dbug_be_memtable_submit_flush_error") } catch (Exception e){ logger.info(e.getMessage()) assertTrue(e.getMessage().contains("dbug_be_memtable_submit_flush_error")) diff --git a/regression-test/suites/fault_injection_p0/test_multi_replica_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_multi_replica_fault_injection.groovy index 2f6afd5ca6925b..d09983d52d0dc3 100644 --- a/regression-test/suites/fault_injection_p0/test_multi_replica_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_multi_replica_fault_injection.groovy @@ -75,14 +75,15 @@ suite("test_multi_replica_fault_injection", "nonConcurrent") { file "baseall.txt" } - def load_with_injection = { injection, error_msg-> + def load_with_injection = { injection, error_msg, success=false-> try { sql "truncate table test" GetDebugPoint().enableDebugPointForAllBEs(injection) sql "insert into test select * from baseall where k1 <= 3" + assertTrue(success, String.format("Expected Exception '%s', actual success", error_msg)) } catch(Exception e) { logger.info(e.getMessage()) - assertTrue(e.getMessage().contains(error_msg)) + assertTrue(e.getMessage().contains(error_msg), e.toString()) } finally { GetDebugPoint().disableDebugPointForAllBEs(injection) } @@ -90,15 +91,17 @@ suite("test_multi_replica_fault_injection", "nonConcurrent") { // StreamSinkFileWriter appendv write segment failed one replica // success - load_with_injection("StreamSinkFileWriter.appendv.write_segment_failed_one_replica", "sucess") + load_with_injection("StreamSinkFileWriter.appendv.write_segment_failed_one_replica", "sucess", true) // StreamSinkFileWriter appendv write segment failed two replica load_with_injection("StreamSinkFileWriter.appendv.write_segment_failed_two_replica", "add segment failed") // StreamSinkFileWriter appendv write segment failed all replica load_with_injection("StreamSinkFileWriter.appendv.write_segment_failed_all_replica", "failed to send segment data to any replicas") // test segment num check when LoadStreamStub missed tail segments - load_with_injection("LoadStreamStub.only_send_segment_0", "segment num mismatch") + load_with_injection("LoadStreamStub.skip_send_segment", "segment num mismatch") // test one backend open failure - load_with_injection("VTabletWriterV2._open_streams.skip_one_backend", "success") + load_with_injection("VTabletWriterV2._open_streams.skip_one_backend", "success", true) + // test two backend open failure + load_with_injection("VTabletWriterV2._open_streams.skip_two_backends", "not enough streams 1/3") sql """ set enable_memtable_on_sink_node=false """ } } diff --git a/regression-test/suites/fault_injection_p0/test_rowset_writer_fault.groovy b/regression-test/suites/fault_injection_p0/test_rowset_writer_fault.groovy index 005e4b6bc974d8..0e5113f2a6ed14 100644 --- a/regression-test/suites/fault_injection_p0/test_rowset_writer_fault.groovy +++ b/regression-test/suites/fault_injection_p0/test_rowset_writer_fault.groovy @@ -56,11 +56,8 @@ suite("test_rowset_writer_fault", "nonConcurrent") { assertEquals("fail", json.Status.toLowerCase()) } } - } catch(Exception e) { - logger.info(e.getMessage()) - assertTrue(e.getMessage().contains(error_msg)) } finally { GetDebugPoint().disableDebugPointForAllBEs(injection) } sql """ DROP TABLE IF EXISTS `baseall` """ -} \ No newline at end of file +} diff --git a/regression-test/suites/fault_injection_p0/test_writer_v2_fault_injection.groovy b/regression-test/suites/fault_injection_p0/test_writer_v2_fault_injection.groovy index f05a084035cb3f..7e71de173e96f9 100644 --- a/regression-test/suites/fault_injection_p0/test_writer_v2_fault_injection.groovy +++ b/regression-test/suites/fault_injection_p0/test_writer_v2_fault_injection.groovy @@ -67,10 +67,11 @@ suite("test_writer_v2_fault_injection", "nonConcurrent") { file "baseall.txt" } - def load_with_injection = { injection, error_msg-> + def load_with_injection = { injection, error_msg, success=false-> try { GetDebugPoint().enableDebugPointForAllBEs(injection) sql "insert into test select * from baseall where k1 <= 3" + assertTrue(success, String.format("expected Exception '%s', actual success", error_msg)) } catch(Exception e) { logger.info(e.getMessage()) assertTrue(e.getMessage().contains(error_msg), @@ -88,7 +89,7 @@ suite("test_writer_v2_fault_injection", "nonConcurrent") { // VTabletWriterV2 node_info is null load_with_injection("VTabletWriterV2._open_streams_to_backend.node_info_null", "failed to open streams to any BE") // VTabletWriterV2 do not get tablet schema on open_streams - load_with_injection("VTabletWriterV2._open_streams_to_backend.no_schema_when_open_streams", "success") + load_with_injection("VTabletWriterV2._open_streams_to_backend.no_schema_when_open_streams", "success", true) // VTabletWriterV2 tablet_location is null load_with_injection("VTabletWriterV2._build_tablet_node_mapping.tablet_location_null", "unknown tablet location") // VTabletWriterV2 location is null diff --git a/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy b/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy index e2ab9b9c117f1c..cce6ed9fd9d014 100644 --- a/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy +++ b/regression-test/suites/index_p0/test_ngram_bloomfilter_index.groovy @@ -104,6 +104,10 @@ suite("test_ngram_bloomfilter_index") { """ test { sql """ALTER TABLE ${tableName3} ADD INDEX idx_http_url(http_url) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="65536") COMMENT 'http_url ngram_bf index'""" - exception "bf_size should be integer and between 64 and 65535" + exception "'bf_size' should be an integer between 64 and 65535" + } + test { + sql """ALTER TABLE ${tableName3} ADD INDEX idx_http_url(http_url) USING NGRAM_BF PROPERTIES("gram_size"="256", "bf_size"="65535") COMMENT 'http_url ngram_bf index'""" + exception "'gram_size' should be an integer between 1 and 255" } } diff --git a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_dup_keys_arr.groovy b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_dup_keys_arr.groovy index ba30c9f044b7be..c7d41e68cdd112 100644 --- a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_dup_keys_arr.groovy +++ b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_dup_keys_arr.groovy @@ -17,7 +17,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods -suite("test_index_compaction_dup_keys_array", "array_contains_inverted_index") { +suite("test_index_compaction_dup_keys_array", "array_contains_inverted_index, nonConcurrent") { // here some variable to control inverted index query sql """ set enable_profile=true""" sql """ set enable_pipeline_x_engine=true;""" diff --git a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_null_arr.groovy b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_null_arr.groovy index b33a197bd32170..dfc38998c4d4bc 100644 --- a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_null_arr.groovy +++ b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_null_arr.groovy @@ -17,7 +17,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods -suite("test_index_compaction_null_arr", "array_contains_inverted_index") { +suite("test_index_compaction_null_arr", "array_contains_inverted_index, nonConcurrent") { // here some variable to control inverted index query sql """ set enable_profile=true""" sql """ set enable_pipeline_x_engine=true;""" diff --git a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy index f8bf4752a2a65e..19162463707c40 100644 --- a/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy +++ b/regression-test/suites/inverted_index_p0/array_contains/test_index_compaction_unique_keys_arr.groovy @@ -17,7 +17,7 @@ import org.codehaus.groovy.runtime.IOGroovyMethods -suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index") { +suite("test_index_compaction_unique_keys_arr", "array_contains_inverted_index, nonConcurrent") { // here some variable to control inverted index query sql """ set enable_profile=true""" sql """ set enable_pipeline_x_engine=true;""" diff --git a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_empty_segments.groovy b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_empty_segments.groovy index 46126299bf5b88..1c70c9e8e5027b 100644 --- a/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_empty_segments.groovy +++ b/regression-test/suites/inverted_index_p0/index_compaction/test_index_compaction_empty_segments.groovy @@ -19,7 +19,7 @@ import java.util.concurrent.Executors import java.util.concurrent.TimeUnit import org.awaitility.Awaitility -suite("test_index_compaction_empty_segments", "p0") { +suite("test_index_compaction_empty_segments", "p0, nonConcurrent") { def compaction_table_name = "test_index_compaction_empty_segments" def backendId_to_backendIP = [:] @@ -82,8 +82,9 @@ suite("test_index_compaction_empty_segments", "p0") { }); } - int afterSegmentCount = 0 + for (def tablet in tablets) { + int afterSegmentCount = 0 String tablet_id = tablet.TabletId (code, out, err) = curl("GET", tablet.CompactionStatus) logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err) @@ -94,6 +95,6 @@ suite("test_index_compaction_empty_segments", "p0") { logger.info("rowset is: " + rowset) afterSegmentCount += Integer.parseInt(rowset.split(" ")[1]) } + assertEquals(afterSegmentCount, 0) } - assertEquals(afterSegmentCount, 0) } diff --git a/regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.groovy b/regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.groovy new file mode 100644 index 00000000000000..ced1c8d74aecfc --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_inverted_index_writer_exception.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. + +import java.sql.SQLException + +suite("test_inverted_index_writer_exception", "nonConcurrent"){ + def indexTbName1 = "test_inverted_index_writer_exception" + + sql "DROP TABLE IF EXISTS ${indexTbName1}" + + sql """ + CREATE TABLE ${indexTbName1} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + try { + GetDebugPoint().enableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + try { + sql """ INSERT INTO ${indexTbName1} VALUES (1, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + } catch (SQLException e) { + if (e.message.contains("E-6002")) { + log.info("Test passed 1: Encountered expected exception [E-6002].") + } else { + throw e + } + } + } finally { + GetDebugPoint().disableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + } + + try { + GetDebugPoint().enableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + try { + sql """ INSERT INTO ${indexTbName1} VALUES (2, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + } catch (SQLException e) { + if (e.message.contains("E-6002")) { + log.info("Test passed 2: Encountered expected exception [E-6002].") + } else { + throw e + } + } + } finally { + GetDebugPoint().disableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + } + + try { + GetDebugPoint().enableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + GetDebugPoint().enableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + + try { + sql """ INSERT INTO ${indexTbName1} VALUES (3, "40.135.0.0", "GET /images/hm_bg.jpg HTTP/1.0", 1, 2); """ + } catch (SQLException e) { + if (e.message.contains("E-6002")) { + log.info("Test passed 3: Encountered expected exception [E-6002].") + } else { + throw e + } + } + } finally { + GetDebugPoint().disableDebugPointForAllBEs("InvertedIndexWriter._throw_clucene_error_in_fulltext_writer_close") + GetDebugPoint().disableDebugPointForAllBEs("DorisFSDirectory::close_close_with_error") + } +} \ No newline at end of file diff --git a/regression-test/suites/job_p0/test_base_insert_job.groovy b/regression-test/suites/job_p0/test_base_insert_job.groovy index 8a0bb34ca43fd5..fc97e367483b86 100644 --- a/regression-test/suites/job_p0/test_base_insert_job.groovy +++ b/regression-test/suites/job_p0/test_base_insert_job.groovy @@ -76,6 +76,7 @@ suite("test_base_insert_job") { insert into ${tableName} values ('2023-03-18', 1, 1) """ + // create recurring job sql """ CREATE JOB ${jobName} ON SCHEDULE every 1 second comment 'test' DO INSERT INTO ${tableName} (`timestamp`, `type`, `user_id`) WITH @@ -97,23 +98,25 @@ suite("test_base_insert_job") { """ Awaitility.await().atMost(30, SECONDS).until( { - def onceJob = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='RECURRING' """ - println(onceJob) - onceJob.size() == 1 && '1' <= onceJob.get(0).get(0) + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='RECURRING' """ + // check job status and succeed task count larger than 1 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) } ) sql """ PAUSE JOB where jobname = '${jobName}' """ + def pausedJobStatus = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + assert pausedJobStatus.get(0).get(0) == "PAUSED" def tblDatas = sql """select * from ${tableName}""" - println tblDatas assert tblDatas.size() >= 2 //at least 2 records - def pauseJobId = sql """select id from jobs("type"="insert") where Name='${jobName}'""" - def taskStatus = sql """select status from tasks("type"="insert") where jobid= '${pauseJobId.get(0).get(0)}'""" - println taskStatus + + def taskStatus = sql """select status from tasks("type"="insert") where JobName ='${jobName}'""" for (int i = 0; i < taskStatus.size(); i++) { - assert taskStatus.get(i).get(0) != "FAILED" || taskStatus.get(i).get(0) != "STOPPED" || taskStatus.get(i).get(0) != "STOPPED" + assert taskStatus.get(i).get(0) =="CANCELLED" || taskStatus.get(i).get(0) =="FINISHED" } sql """ CREATE JOB ${jobMixedName} ON SCHEDULE every 1 second DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); @@ -122,6 +125,7 @@ suite("test_base_insert_job") { println mixedNameJobs assert mixedNameJobs.size() == 1 && mixedNameJobs.get(0).get(0) == jobMixedName assert mixedNameJobs.get(0).get(1) == '' + // clean up job and table sql """ DROP JOB IF EXISTS where jobname = '${jobName}' """ @@ -145,23 +149,25 @@ suite("test_base_insert_job") { """ def dataCount = sql """select count(*) from ${tableName}""" assert dataCount.get(0).get(0) == 0 + // create one time job sql """ CREATE JOB ${jobName} ON SCHEDULE at current_timestamp comment 'test for test&68686781jbjbhj//ncsa' DO insert into ${tableName} values ('2023-07-19', 2, 1001); """ - + // wait job finished Awaitility.await("create-one-time-job-test").atMost(30, SECONDS).until( { def onceJob = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='ONE_TIME' """ onceJob.size() == 1 && '1' == onceJob.get(0).get(0) } ) - def onceJob = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='ONE_TIME' """ + def onceJob = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='ONE_TIME' """ assert onceJob.size() == 1 //check succeed task count assert '1' == onceJob.get(0).get(0) def datas = sql """select status,taskid from tasks("type"="insert") where jobName= '${jobName}'""" - println datas + // table should have one record after job finished assert datas.size() == 1 + // one time job only has one task. when job finished, task status should be FINISHED assert datas.get(0).get(0) == "FINISHED" // check table data def dataCount1 = sql """select count(1) from ${tableName} where user_id=1001""" @@ -175,19 +181,20 @@ suite("test_base_insert_job") { sql """ DROP JOB IF EXISTS where jobname = 'press' """ - + // create job with start time is current time and interval is 10 hours sql """ CREATE JOB press ON SCHEDULE every 10 hour starts CURRENT_TIMESTAMP comment 'test for test&68686781jbjbhj//ncsa' DO insert into ${tableName} values ('2023-07-19', 99, 99); """ Awaitility.await("create-immediately-job-test").atMost(60, SECONDS).until({ def pressJob = sql """ select SucceedTaskCount from jobs("type"="insert") where name='press'""" - println pressJob + // check job status and succeed task count is 1 pressJob.size() == 1 && '1' == onceJob.get(0).get(0) }) sql """ DROP JOB IF EXISTS where jobname = 'past_start_time' """ + // create job with start time is past time, job should be running sql """ CREATE JOB past_start_time ON SCHEDULE every 10 hour starts '2023-11-13 14:18:07' comment 'test for test&68686781jbjbhj//ncsa' DO insert into ${tableName} values ('2023-07-19', 99, 99); """ @@ -214,6 +221,10 @@ suite("test_base_insert_job") { sql """ PAUSE JOB where jobname = '${jobName}' """ + pausedJobStatus = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + assert pausedJobStatus.get(0).get(0) == "PAUSED" def tasks = sql """ select status from tasks("type"="insert") where JobName= '${jobName}' """ sql """ RESUME JOB where jobname = '${jobName}' @@ -226,6 +237,11 @@ suite("test_base_insert_job") { //resume tasks size should be greater than before pause afterResumeTasks.size() > tasks.size() }) + // check resume job status + def afterResumeJobStatus = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + assert afterResumeJobStatus.get(0).get(0) == "RUNNING" // assert same job name try { diff --git a/regression-test/suites/manager/test_manager_interface_1.groovy b/regression-test/suites/manager/test_manager_interface_1.groovy index 249ce252ffbf86..02f6c210f1fd09 100644 --- a/regression-test/suites/manager/test_manager_interface_1.groovy +++ b/regression-test/suites/manager/test_manager_interface_1.groovy @@ -496,128 +496,6 @@ DISTRIBUTED BY HASH(`k1`) BUCKETS 1""")) test_table_index() - - - -// show proc '/current_query_stmts' -// show proc '/current_queries' -// show processlist -// kill query $query_id -// SHOW PROC '/cluster_health/tablet_health' - def test_proc = { - - def futures = [] - - - futures.add( thread { - - try{ - sql """ select sleep(4.809); """ - }catch(Exception e){ - - } - }) - futures.add( thread { - sleep(1000); - List> result = sql """ show proc '/current_query_stmts' """ - logger.info("result = ${result}" ) - def x = 0 - def queryid = "" - logger.info("result = ${result}") - - for( int i = 0;i> result = sql """ show proc '/current_query_stmts' """ + logger.info("result = ${result}" ) + def x = 0 + def queryid = "" + logger.info("result = ${result}") + + for( int i = 0;i= 1 AND LO_DISCOUNT <= 3 AND LO_QUANTITY < 25 - GROUP BY - LO_ORDERKEY;""") + 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');""" - + 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'), + (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'), + (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'), + (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');""" qt_select_star "select * from lineorder_flat order by 1,2, P_MFGR;" sql "analyze table lineorder_flat with sync;" diff --git a/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy b/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy index 6ead8380eed59b..0ac5f0245ef8ae 100644 --- a/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_2_1/q_2_1.groovy @@ -93,6 +93,8 @@ suite ("mv_ssb_q_2_1") { qt_select_star "select * from lineorder_flat order by 1,2,P_MFGR;" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='2');""" + mv_rewrite_success("""SELECT SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND diff --git a/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy b/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy index 2b170fe3b2fefd..6b333d97050630 100644 --- a/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_3_1/q_3_1.groovy @@ -98,6 +98,8 @@ suite ("mv_ssb_q_3_1") { sql """analyze table lineorder_flat with sync;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='2');""" + mv_rewrite_success("""SELECT C_NATION, S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, diff --git a/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy b/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy index c094e8e00c9eaf..361f60997e3da6 100644 --- a/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_4_1/q_4_1.groovy @@ -92,6 +92,8 @@ suite ("mv_ssb_q_4_1") { sql """analyze table lineorder_flat with sync;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='2');""" + mv_rewrite_success("""SELECT (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy b/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy index 1baf21b547f062..ebeb23d96e75e5 100644 --- a/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_4_1_r1/q_4_1_r1.groovy @@ -119,6 +119,7 @@ suite ("q_4_1_r1") { GROUP BY YEAR, C_NATION ORDER BY YEAR ASC, C_NATION ASC;""" sql """set enable_stats=true;""" + sql """alter table lineorder_flat modify column LO_ORDERDATE set stats ('row_count'='8');""" mv_rewrite_success("""SELECT (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/mv_p0/sum_count/sum_count.groovy b/regression-test/suites/mv_p0/sum_count/sum_count.groovy index b47e51999b7c42..137aabc4e16a89 100644 --- a/regression-test/suites/mv_p0/sum_count/sum_count.groovy +++ b/regression-test/suites/mv_p0/sum_count/sum_count.groovy @@ -51,6 +51,7 @@ suite ("sum_count") { sql "analyze table d_table with sync;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='8');""" mv_rewrite_success("select k1,k4,sum(k2),count(k2) from d_table group by k1,k4;", "kavg") qt_select_mv "select k1,k4,sum(k2),count(k2) from d_table group by k1,k4 order by 1,2;" diff --git a/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy b/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy index 4e6374e300b09e..85a3f7137b9e40 100644 --- a/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy +++ b/regression-test/suites/mv_p0/sum_divede_count/sum_devide_count.groovy @@ -46,6 +46,7 @@ suite ("sum_devide_count") { sql """analyze table d_table with sync;""" sql """set enable_stats=false;""" + mv_rewrite_success("select k1,k4,sum(k2)/count(k2) from d_table group by k1,k4 order by k1,k4;", "kavg") qt_select_mv "select k1,k4,sum(k2)/count(k2) from d_table group by k1,k4 order by k1,k4;" @@ -59,6 +60,7 @@ suite ("sum_devide_count") { qt_select_mv "select sum(k2)/count(k2) from d_table;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k4,sum(k2)/count(k2) from d_table group by k1,k4 order by k1,k4;", "kavg") mv_rewrite_success("select k1,sum(k2)/count(k2) from d_table group by k1 order by k1;", "kavg") diff --git a/regression-test/suites/mv_p0/test_28741/test_28741.groovy b/regression-test/suites/mv_p0/test_28741/test_28741.groovy index 9fabca41b943fc..88fb3c057da092 100644 --- a/regression-test/suites/mv_p0/test_28741/test_28741.groovy +++ b/regression-test/suites/mv_p0/test_28741/test_28741.groovy @@ -69,8 +69,8 @@ suite ("test_28741") { sql """set enable_stats=false;""" mv_rewrite_fail("select b1 from test where t >= '2023-12-20 17:21:00'", "mv_test") - qt_select "select b1 from test where t >= '2023-12-20 17:21:00'" sql """set enable_stats=true;""" + sql """alter table test modify column a set stats ('row_count'='2');""" mv_rewrite_fail("select b1 from test where t >= '2023-12-20 17:21:00'", "mv_test") } diff --git a/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy b/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy index 752cf40d9e5f6a..68952da602aa81 100644 --- a/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy +++ b/regression-test/suites/mv_p0/test_approx_count_distinct/test_approx_count_distinct.groovy @@ -50,6 +50,7 @@ suite ("test_approx_count_distinct") { qt_select_mv "select user_id, approx_count_distinct(tag_id) a from user_tags group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, ndv(tag_id) a from user_tags group by user_id order by user_id;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/test_base/test_base.groovy b/regression-test/suites/mv_p0/test_base/test_base.groovy index d2c018febd5ada..97ece8cd6cca68 100644 --- a/regression-test/suites/mv_p0/test_base/test_base.groovy +++ b/regression-test/suites/mv_p0/test_base/test_base.groovy @@ -53,6 +53,7 @@ suite ("test_base") { qt_select_mv "SELECT id,created_at FROM dwd order by 1, 2;" sql """set enable_stats=true;""" + sql """alter table dwd modify column id set stats ('row_count'='2');""" mv_rewrite_success("SELECT created_at, id FROM dwd where created_at = '2020-09-09 00:00:00' order by 1, 2;", "dwd_mv") mv_rewrite_success("SELECT id,created_at FROM dwd where id is not null order by 1, 2;", "dwd_mv") diff --git a/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy b/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy index eac65eeef22adf..0c9e0026cb7fbd 100644 --- a/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy +++ b/regression-test/suites/mv_p0/test_casewhen/test_casewhen.groovy @@ -40,5 +40,6 @@ suite ("test_casewhen") { qt_select_mv "select store_id, sum(case when sale_amt>10 then 1 else 2 end) from sales_records group by store_id order by 1;" sql """set enable_stats=true;""" + sql """alter table sales_records modify column record_id set stats ('row_count'='4');""" mv_rewrite_success("select store_id, sum(case when sale_amt>10 then 1 else 2 end) from sales_records group by store_id order by 1;", "store_amt") } diff --git a/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy b/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy index 776bf2ecf0bd17..8ee24cb5ba2bd9 100644 --- a/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy +++ b/regression-test/suites/mv_p0/test_create_mv/test_create_mv.groovy @@ -49,6 +49,8 @@ suite("test_create_mv") { sql """ insert into ${tableName} values ('2024-03-20 10:00:00', 'a', 'b', 1) """ + sql """alter table test_mv_10010 modify column load_time set stats ('row_count'='1');""" + sql """ create materialized view mv_1 as select diff --git a/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy b/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy index 12790500d2f98e..1e07d69ba82871 100644 --- a/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy +++ b/regression-test/suites/mv_p0/test_create_mv_complex_type/test_create_mv_complex_type.groovy @@ -37,6 +37,8 @@ suite ("create_mv_complex_type") { sql """insert into base_table select 1, 100000, 1.0, '{"jsonk1": 123}', [100, 200], {"k1": 10}, {1, 2};""" + sql """alter table base_table modify column c_int set stats ('row_count'='1');""" + def success = false // 1. special column - mv dup key diff --git a/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy b/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy index 5e99464018e8d7..517b4415bd69ea 100644 --- a/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy +++ b/regression-test/suites/mv_p0/test_doc_e4/test_doc_e4.groovy @@ -57,6 +57,7 @@ suite ("test_doc_e4") { qt_select_mv "select year(k4)+month(k4) from d_table where year(k4) = 2020 order by 1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='3');""" mv_rewrite_success("select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from d_table group by abs(k1)+k2+1 order by 1,2;", "k1a2p2ap3ps") mv_rewrite_success("select bin(abs(k1)+k2+1),sum(abs(k2+2)+k3+3) from d_table group by bin(abs(k1)+k2+1);", "k1a2p2ap3ps") diff --git a/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy b/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy index 5e917e9b0f5bd0..23aeeb207f6898 100644 --- a/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy +++ b/regression-test/suites/mv_p0/test_dup_group_by_mv_abs/test_dup_group_by_mv_abs.groovy @@ -53,6 +53,7 @@ suite ("test_dup_group_by_mv_abs") { qt_select_mv_sub "select sum(abs(k2)) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(abs(k2)) from d_table group by k1;", "k12sa") mv_rewrite_success("select sum(abs(k2)) from d_table group by k1;", "k12sa") } diff --git a/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy b/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy index ba9f60f6df5026..3551bb8a7fb359 100644 --- a/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy +++ b/regression-test/suites/mv_p0/test_dup_group_by_mv_plus/test_dup_group_by_mv_plus.groovy @@ -53,6 +53,7 @@ suite ("test_dup_group_by_mv_plus") { qt_select_mv_sub "select sum(k2+1) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(k2+1) from d_table group by k1;", "k12sp") mv_rewrite_success("select sum(k2+1) from d_table group by k1;", "k12sp") diff --git a/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy b/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy index 89088ffb9ea317..f44c9d134f91f0 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_abs/test_dup_mv_abs.groovy @@ -65,6 +65,7 @@ suite ("test_dup_mv_abs") { qt_select_group_mv_not "select sum(abs(k2)) from d_table group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,abs(k2) from d_table order by k1;", "k12a") mv_rewrite_success("select abs(k2) from d_table order by k1;", "k12a") diff --git a/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy b/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy index a7db2617fe6067..4b8fa5aa6359e8 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_bin/test_dup_mv_bin.groovy @@ -64,6 +64,7 @@ suite ("test_dup_mv_bin") { qt_select_group_mv_not "select group_concat(bin(k2)) from d_table group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,bin(k2) from d_table order by k1;", "k12b") mv_rewrite_success("select bin(k2) from d_table order by k1;", "k12b") diff --git a/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy b/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy index 659f44de1e93c4..6530fd84cac7a8 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_bitmap_hash/test_dup_mv_bitmap_hash.groovy @@ -46,6 +46,7 @@ suite ("test_dup_mv_bitmap_hash") { mv_rewrite_success("select bitmap_union_count(to_bitmap(k2)) from d_table group by k1 order by k1;", "k1g2bm") qt_select_mv "select bitmap_union_count(to_bitmap(k2)) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select bitmap_union_count(to_bitmap(k2)) from d_table group by k1 order by k1;", "k1g2bm") createMV "create materialized view k1g3bm as select k1,bitmap_union(bitmap_hash(k3)) from d_table group by k1;" @@ -58,11 +59,13 @@ suite ("test_dup_mv_bitmap_hash") { qt_select_star "select * from d_table order by k1,k2,k3;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" sql """analyze table d_table with sync;""" sql """set enable_stats=false;""" mv_rewrite_success("select k1,bitmap_union_count(bitmap_hash(k3)) from d_table group by k1;", "k1g3bm") qt_select_mv_sub "select k1,bitmap_union_count(bitmap_hash(k3)) from d_table group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,bitmap_union_count(bitmap_hash(k3)) from d_table group by k1;", "k1g3bm") } diff --git a/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy b/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy index 5abe03e736b641..ad437db464b3bb 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_plus/test_dup_mv_plus.groovy @@ -70,7 +70,7 @@ suite ("test_dup_mv_plus") { qt_select_mv "select k1,k2+1 from d_table order by k2;" sql """set enable_stats=true;""" - + sql """alter table d_table modify column k4 set stats ('row_count'='3');""" mv_rewrite_success("select k1,k2+1 from d_table order by k1;", "k12p") mv_rewrite_success("select k2+1 from d_table order by k1;", "k12p") diff --git a/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy b/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy index 562fba25b2e8a9..5ccdae1bd8a7c5 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_repeat/test_dup_mv_repeat.groovy @@ -47,6 +47,7 @@ suite ("test_dup_mv_repeat") { qt_select_mv "SELECT s AS s, sum(n) / count(DISTINCT dt) AS n FROM db1 GROUP BY GROUPING SETS((s)) order by 1;" sql """set enable_stats=true;""" + sql """alter table db1 modify column dt set stats ('row_count'='2');""" mv_rewrite_success("SELECT s AS s, sum(n) / count(DISTINCT dt) AS n FROM db1 GROUP BY GROUPING SETS((s)) order by 1;", "dbviwe") } diff --git a/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy b/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy index 92a8d6520f3122..18f798f8b6aac3 100644 --- a/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy +++ b/regression-test/suites/mv_p0/test_dup_mv_year/test_dup_mv_year.groovy @@ -45,6 +45,7 @@ suite ("test_dup_mv_year") { qt_select_mv "select k1,year(k2) from d_table order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,year(k2) from d_table order by k1;", "k12y") createMV "create materialized view k13y as select k1,year(k3) from d_table;" diff --git a/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy b/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy index ccd87d792348c3..c1a495a74e20bc 100644 --- a/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy +++ b/regression-test/suites/mv_p0/test_duplicate_mv/test_duplicate_mv.groovy @@ -39,6 +39,8 @@ suite ("test_duplicate_mv") { sql "insert into duplicate_table select 2,3,4,5;" sql "insert into duplicate_table select 1,2,3,4;" + sql """alter table duplicate_table modify column k1 set stats ('row_count'='3');""" + qt_select_star "select * from duplicate_table order by k1;" mv_rewrite_success("select k1, k2, k3, k4 from duplicate_table group by k1, k2, k3, k4;", "deduplicate") diff --git a/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy b/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy index 50f1fd61806f2e..2fadc0b0da91a0 100644 --- a/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy +++ b/regression-test/suites/mv_p0/test_mv_dp/test_mv_dp.groovy @@ -56,6 +56,7 @@ suite ("test_mv_dp") { time 10000 // limit inflight 10s } */ + mv_rewrite_success("""select d, bitmap_union_count(bitmap_from_array(cast(uid_list as array))), bitmap_union_count(bitmap_from_array(if(status='success', cast(uid_list as array), array()))) @@ -68,6 +69,7 @@ suite ("test_mv_dp") { from dp group by d order by 1;""" sql """set enable_stats=true;""" + sql """alter table dp modify column d set stats ('row_count'='4');""" mv_rewrite_success("""select d, bitmap_union_count(bitmap_from_array(cast(uid_list as array))), bitmap_union_count(bitmap_from_array(if(status='success', cast(uid_list as array), array()))) diff --git a/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy b/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy index b7f95123f5f9dd..072c7d905cbea0 100644 --- a/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy +++ b/regression-test/suites/mv_p0/test_mv_mor/test_mv_mor.groovy @@ -39,6 +39,8 @@ suite ("test_mv_mor") { sql "insert into u_table select 1,1,1,2;" sql "insert into u_table select 1,2,1,2;" + sql """alter table u_table modify column k1 set stats ('row_count'='2');""" + // do not match mv coz preagg is off, mv need contains all key column to make row count correct mv_rewrite_success("select k1,k2+k3 from u_table order by k1;", "k123p") qt_select_mv "select k1,k2+k3 from u_table order by k1;" diff --git a/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy b/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy index f584730ad1fca3..3fe10c2e50a33a 100644 --- a/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy +++ b/regression-test/suites/mv_p0/test_mv_mow/test_mv_mow.groovy @@ -50,5 +50,6 @@ suite ("test_mv_mow") { qt_select_mv "select mv_k1 from `u_table` index `k123p` order by 1;" qt_select_mv "select `mv_(k2 + k3)` from `u_table` index `k123p` order by 1;" sql """set enable_stats=true;""" + sql """alter table u_table modify column k1 set stats ('row_count'='2');""" mv_rewrite_success("select k1,k2+k3 from u_table order by k1;", "k123p") } diff --git a/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy b/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy index d2c9921edefe7a..2d43785ba6fafb 100644 --- a/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy +++ b/regression-test/suites/mv_p0/test_ndv/test_ndv.groovy @@ -50,6 +50,7 @@ suite ("test_ndv") { qt_select_mv "select user_id, approx_count_distinct(tag_id) a from user_tags group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, ndv(tag_id) a from user_tags group by user_id order by user_id;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy b/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy index c6b1dd2d8a9b1c..12f5765666ef28 100644 --- a/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy +++ b/regression-test/suites/mv_p0/test_nvl/test_nvl.groovy @@ -51,6 +51,7 @@ suite ("test_nvl") { qt_select_mv "select ifnull(id,0) from dwd order by 1;" sql """set enable_stats=true;""" + sql """alter table dwd modify column id set stats ('row_count'='2');""" mv_rewrite_success("select nvl(id,0) from dwd order by 1;", "dwd_mv") mv_rewrite_success("select ifnull(id,0) from dwd order by 1;", "dwd_mv") diff --git a/regression-test/suites/mv_p0/test_o2/test_o2.groovy b/regression-test/suites/mv_p0/test_o2/test_o2.groovy index 665156056c0096..34d31a08cfae7b 100644 --- a/regression-test/suites/mv_p0/test_o2/test_o2.groovy +++ b/regression-test/suites/mv_p0/test_o2/test_o2.groovy @@ -60,6 +60,7 @@ suite ("test_o2") { qt_select_mv "select ts,metric_name,platform,sum(count_value) from o2_order_events group by ts,metric_name,platform;" sql """set enable_stats=true;""" + sql """alter table o2_order_events modify column ts set stats ('row_count'='2');""" mv_rewrite_success("select ts,metric_name,platform,sum(count_value) from o2_order_events group by ts,metric_name,platform;", "o2_order_events_mv") } diff --git a/regression-test/suites/mv_p0/test_substr/test_substr.groovy b/regression-test/suites/mv_p0/test_substr/test_substr.groovy index c43c3ae6483acd..7d6ae82634fd8c 100644 --- a/regression-test/suites/mv_p0/test_substr/test_substr.groovy +++ b/regression-test/suites/mv_p0/test_substr/test_substr.groovy @@ -56,6 +56,7 @@ suite ("test_substr") { qt_select_mv "SELECT substr(created_at,1,10) as statistic_date, max(dt) as dt FROM dwd group by substr(created_at,1,10);" sql """set enable_stats=true;""" + sql """alter table dwd modify column id set stats ('row_count'='2');""" mv_rewrite_success("SELECT substr(created_at,1,10) as statistic_date, max(dt) as dt FROM dwd group by substr(created_at,1,10);", "dwd_mv") } diff --git a/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy b/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy index f65e36484e7de9..bbfa8ea6a54d17 100644 --- a/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy +++ b/regression-test/suites/mv_p0/test_tbl_name/test_tbl_name.groovy @@ -66,6 +66,7 @@ suite ("test_tbl_name") { group by id order by 1,2; """ sql """set enable_stats=true;""" + sql """alter table functionality_olap modify column id set stats ('row_count'='2');""" mv_rewrite_success("""select functionality_olap.id as id, sum(functionality_olap.score) as score_max diff --git a/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy b/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy index b62ec696224197..69264d838e4d24 100644 --- a/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy +++ b/regression-test/suites/mv_p0/test_upper_alias/test_upper_alias.groovy @@ -62,6 +62,7 @@ suite ("test_upper_alias") { qt_select_mv "SELECT d_a AS d_b FROM test_0401 order by 1;" sql """set enable_stats=true;""" + sql """alter table test_0401 modify column d_b set stats ('row_count'='3');""" mv_rewrite_any_success("SELECT upper(d_b) AS d_b FROM test_0401 GROUP BY upper(d_b) order by 1;", ["test_0401_mv", "test_0401_mv2"]) diff --git a/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy b/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy index 3d139cbc01bbe7..f971673971a2aa 100644 --- a/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy +++ b/regression-test/suites/mv_p0/test_user_activity/test_user_activity.groovy @@ -53,6 +53,7 @@ suite ("test_user_activity") { qt_select_group_mv "select n_dx, percentile_approx(n_duration, 0.5) as p50, percentile_approx(n_duration, 0.90) as p90 FROM u_axx GROUP BY n_dx;" sql """set enable_stats=true;""" + sql """alter table u_axx modify column r_xx set stats ('row_count'='3');""" mv_rewrite_success("select n_dx, percentile_approx(n_duration, 0.5) as p50, percentile_approx(n_duration, 0.90) as p90 FROM u_axx GROUP BY n_dx;", "session_distribution_2") } diff --git a/regression-test/suites/mv_p0/unique/unique.groovy b/regression-test/suites/mv_p0/unique/unique.groovy index e2a78c60bc2b0f..d054803678da4f 100644 --- a/regression-test/suites/mv_p0/unique/unique.groovy +++ b/regression-test/suites/mv_p0/unique/unique.groovy @@ -73,6 +73,7 @@ suite ("unique") { qt_select_star "select * from u_table order by k1;" sql """set enable_stats=true;""" + sql """alter table u_table modify column k1 set stats ('row_count'='3');""" mv_rewrite_success("select k3,length(k1),k2 from u_table order by 1,2,3;", "k31l42") // todo: support match query diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy index 70b016de2da80f..f39e8df6cabe6a 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV1/testAggQueryOnAggMV1.groovy @@ -36,6 +36,7 @@ suite ("testAggQueryOnAggMV1") { sql """insert into emps values("2020-01-02",2,"b",2,2,2);""" sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" +sql """alter table emps modify column time_col set stats ('row_count'='4');""" createMV("create materialized view emps_mv as select deptno, sum(salary), max(commission) from emps group by deptno;") createMV("create materialized view emps_mv_count_key as select deptno, count(deptno) from emps group by deptno;") diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy index a25ac24404e181..739dde33372cb6 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV10/testAggQueryOnAggMV10.groovy @@ -51,6 +51,7 @@ suite ("testAggQueryOnAggMV10") { qt_select_mv "select deptno, commission, sum(salary) + 1 from emps group by rollup (deptno, commission) order by 1,2;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select deptno, commission, sum(salary) + 1 from emps group by rollup (deptno, commission);", diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy index c26db8e034e66a..a4d69136bc03ca 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV11/testAggQueryOnAggMV11.groovy @@ -50,6 +50,7 @@ suite ("testAggQueryOnAggMV11") { qt_select_mv "select deptno, count(salary) + count(1) from emps group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_fail("select deptno, count(salary) + count(1) from emps group by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy index 945e4e9194910f..5cf7d257fe8b73 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV2/testAggQueryOnAggMV2.groovy @@ -53,6 +53,7 @@ suite ("testAggQueryOnAggMV2") { qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where (sum_salary * 2) > 3 order by deptno ;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where (sum_salary * 2) > 3 order by deptno ;", diff --git a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy index 46457f0d0a81d4..d2f61e04762b2f 100644 --- a/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQueryOnAggMV3/testAggQueryOnAggMV3.groovy @@ -38,7 +38,6 @@ suite ("testAggQueryOnAggMV3") { sql """insert into emps values("2020-01-04",4,"d",21,4,4);""" - createMV("create materialized view emps_mv as select deptno, commission, sum(salary) from emps group by deptno, commission;") sql "analyze table emps with sync;" @@ -56,6 +55,7 @@ suite ("testAggQueryOnAggMV3") { qt_select_mv "select commission, sum(salary) from emps where commission = 100 group by commission order by commission;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select commission, sum(salary) from emps where deptno > 0 and commission * (deptno + commission) = 100 group by commission order by commission;", diff --git a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy index 1fa4342952cad4..1f14728e3b7b2c 100644 --- a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV5/testAggQuqeryOnAggMV5.groovy @@ -51,6 +51,7 @@ suite ("testAggQuqeryOnAggMV5") { qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where sum_salary>10 order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from emps group by deptno) a where sum_salary>0;", diff --git a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy index 3cca0273051f0a..4cafeaa185f4a3 100644 --- a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV6/testAggQuqeryOnAggMV6.groovy @@ -51,6 +51,7 @@ suite ("testAggQuqeryOnAggMV6") { qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from emps where deptno>=20 group by deptno) a where sum_salary>10 order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from emps where deptno>=0 group by deptno) a where sum_salary>10;", diff --git a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy index 1a606141e38f04..d43b5796e6d7f9 100644 --- a/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy +++ b/regression-test/suites/mv_p0/ut/testAggQuqeryOnAggMV7/testAggQuqeryOnAggMV7.groovy @@ -50,6 +50,8 @@ suite ("testAggQuqeryOnAggMV7") { qt_select_mv "select deptno, sum(salary) from emps where deptno>=20 group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select deptno, sum(salary) from emps where deptno>=20 group by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy b/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy index 7e8199b9932e72..26f1fdb0219a2d 100644 --- a/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy +++ b/regression-test/suites/mv_p0/ut/testAggTableCountDistinctInBitmapType/testAggTableCountDistinctInBitmapType.groovy @@ -42,6 +42,7 @@ suite ("testAggTableCountDistinctInBitmapType") { qt_select_mv "select k1, count(distinct v1) from test_tb group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table test_tb modify column k1 set stats ('row_count'='3');""" explain { sql("select k1, count(distinct v1) from test_tb group by k1;") contains "bitmap_union_count" diff --git a/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy b/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy index 7f2427e275de47..6dc78784956d32 100644 --- a/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy +++ b/regression-test/suites/mv_p0/ut/testAggregateMVCalcAggFunctionQuery/testAggregateMVCalcAggFunctionQuery.groovy @@ -50,6 +50,7 @@ suite ("testAggregateMVCalcAggFunctionQuery") { qt_select_mv "select deptno, sum(salary + 1) from emps where deptno > 10 group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_fail("select deptno, sum(salary + 1) from emps where deptno > 10 group by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy b/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy index 69410ae7be5d0d..c1287bfa94bcc4 100644 --- a/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy +++ b/regression-test/suites/mv_p0/ut/testBitmapUnionInQuery/testBitmapUnionInQuery.groovy @@ -51,7 +51,7 @@ suite ("testBitmapUnionInQuery") { qt_select_mv "select user_id, bitmap_count(bitmap_union(to_bitmap(tag_id))) a from user_tags group by user_id having a>1 order by a;" sql """set enable_stats=true;""" - + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, bitmap_union_count(to_bitmap(tag_id)) a from user_tags group by user_id having a>1 order by a;", diff --git a/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy b/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy index 12152aad369e1d..5cc05db62be1a7 100644 --- a/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy +++ b/regression-test/suites/mv_p0/ut/testCountDistinctToBitmap/testCountDistinctToBitmap.groovy @@ -66,6 +66,10 @@ suite ("testCountDistinctToBitmap") { sql """insert into user_tags2 values("2020-01-01",1,"a",1);""" sql """insert into user_tags2 values("2020-01-02",2,"b",2);""" + + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" + sql """alter table user_tags2 modify column time_col set stats ('row_count'='3');""" + createMV("create materialized view user_tags_mv as select user_id, bitmap_union(to_bitmap(tag_id)) from user_tags2 group by user_id;") sql """insert into user_tags2 values("2020-01-01",1,"a",2);""" diff --git a/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy b/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy index c671e106a10648..698490f0943e88 100644 --- a/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy +++ b/regression-test/suites/mv_p0/ut/testIncorrectMVRewriteInSubquery/testIncorrectMVRewriteInSubquery.groovy @@ -47,6 +47,7 @@ suite ("testIncorrectMVRewriteInSubquery") { qt_select_mv "select user_id, bitmap_union(to_bitmap(tag_id)) from user_tags where user_name in (select user_name from user_tags group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_fail("select user_id, bitmap_union(to_bitmap(tag_id)) from user_tags where user_name in (select user_name from user_tags group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;", diff --git a/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy b/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy index dbece0c9592364..45230c5b5587d8 100644 --- a/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy +++ b/regression-test/suites/mv_p0/ut/testIncorrectRewriteCountDistinct/testIncorrectRewriteCountDistinct.groovy @@ -45,6 +45,7 @@ suite ("testIncorrectRewriteCountDistinct") { qt_select_mv "select user_name, count(distinct tag_id) from user_tags group by user_name order by user_name;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_fail("select user_name, count(distinct tag_id) from user_tags group by user_name;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy b/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy index 3caf0000c1824d..8f41fc1ffb5e74 100644 --- a/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy +++ b/regression-test/suites/mv_p0/ut/testJoinOnLeftProjectToJoin/testJoinOnLeftProjectToJoin.groovy @@ -43,6 +43,8 @@ suite ("testJoinOnLeftProjectToJoin") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table depts modify column time_col set stats ('row_count'='3');""" + sql """insert into depts values("2020-01-02",2,"b",2);""" sql """insert into depts values("2020-01-03",3,"c",3);""" sql """insert into depts values("2020-01-02",2,"b",1);""" @@ -59,6 +61,7 @@ suite ("testJoinOnLeftProjectToJoin") { qt_select_mv "select * from (select deptno , sum(salary) from emps group by deptno) A join (select deptno, max(cost) from depts group by deptno ) B on A.deptno = B.deptno order by A.deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_all_success("select * from (select deptno , sum(salary) from emps group by deptno) A join (select deptno, max(cost) from depts group by deptno ) B on A.deptno = B.deptno;", ["emps_mv", "depts_mv"]) } diff --git a/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy b/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy index a9456cb4e79c96..c77f58609ca0cb 100644 --- a/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy +++ b/regression-test/suites/mv_p0/ut/testNDVToHll/testNDVToHll.groovy @@ -50,6 +50,7 @@ suite ("testNDVToHll") { qt_select_mv "select user_id, approx_count_distinct(tag_id) a from user_tags group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from user_tags order by time_col;", "user_tags_mv") mv_rewrite_success("select user_id, ndv(tag_id) a from user_tags group by user_id order by user_id;", "user_tags_mv") diff --git a/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy b/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy index 2aa88d84f3d433..41cf480e880cd0 100644 --- a/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy +++ b/regression-test/suites/mv_p0/ut/testOrderByQueryOnProjectView/testOrderByQueryOnProjectView.groovy @@ -49,6 +49,7 @@ suite ("testOrderByQueryOnProjectView") { qt_select_mv "select empid from emps order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid from emps where deptno > 0 order by deptno;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy index d95c3e2fc6de88..d578cbddba945d 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV1/testProjectionMV1.groovy @@ -61,6 +61,7 @@ suite ("testProjectionMV1") { qt_select_mv "select deptno, sum(empid) from emps group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid, deptno from emps where deptno > 0 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy index 2f7548b14c8635..b663bfcd2b1738 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV2/testProjectionMV2.groovy @@ -51,6 +51,7 @@ suite ("testProjectionMV2") { qt_select_base "select name from emps where deptno -1 = 0 order by empid;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid + 1 from emps where deptno = 1 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy index e4ea1aa4de023e..378ec80d6cfd72 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV3/testProjectionMV3.groovy @@ -53,6 +53,7 @@ suite ("testProjectionMV3") { qt_select_mv2 "select name from emps where deptno = 1 order by empid;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_success("select empid + 1, name from emps where deptno = 1 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy b/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy index 0c6f737deabb29..37a57f80d03cae 100644 --- a/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy +++ b/regression-test/suites/mv_p0/ut/testProjectionMV4/testProjectionMV4.groovy @@ -52,6 +52,7 @@ suite ("testProjectionMV4") { qt_select_base "select empid from emps where deptno > 1 and empid > 1 order by empid;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") mv_rewrite_fail("select empid from emps where deptno > 1 and empid > 1 order by empid;", "emps_mv") diff --git a/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy b/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy index d81129d520729f..7d563883514f72 100644 --- a/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy +++ b/regression-test/suites/mv_p0/ut/testQueryOnStar/testQueryOnStar.groovy @@ -58,4 +58,5 @@ suite ("testQueryOnStar") { sql """insert into tpch_tiny_region values(1,'a','a');""" qt_select_mv "select ref_1.`empid` as c0 from tpch_tiny_region as ref_0 left join emps as ref_1 on (ref_0.`r_comment` = ref_1.`name` ) where true order by ref_0.`r_regionkey`,ref_0.`r_regionkey` desc ,ref_0.`r_regionkey`,ref_0.`r_regionkey`;" + } diff --git a/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy b/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy index 0cfd80fcb06b4e..9e3bd886b31ea5 100644 --- a/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy +++ b/regression-test/suites/mv_p0/ut/testSelectMVWithTableAlias/testSelectMVWithTableAlias.groovy @@ -29,6 +29,8 @@ suite ("testSelectMVWithTableAlias") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table user_tags modify column time_col set stats ('row_count'='3');""" + sql """insert into user_tags values("2020-01-01",1,"a",1);""" sql """insert into user_tags values("2020-01-02",2,"b",2);""" diff --git a/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy b/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy index 006b72885ec112..73ddb21109bb91 100644 --- a/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy +++ b/regression-test/suites/mv_p0/ut/testSingleMVMultiUsage/testSingleMVMultiUsage.groovy @@ -52,6 +52,7 @@ suite ("testSingleMVMultiUsage") { } qt_select_mv "select * from (select deptno, empid from emps where deptno>100) A join (select deptno, empid from emps where deptno >200) B using (deptno) order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") explain { diff --git a/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy b/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy index 6c1747059d1ae3..cd134a5d48ea40 100644 --- a/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy +++ b/regression-test/suites/mv_p0/ut/testSubQuery/testSubQuery.groovy @@ -38,7 +38,6 @@ suite ("testSubQuery") { sql """insert into emps values("2020-01-02",2,"b",2,2,2);""" sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view emps_mv as select deptno, empid from emps;") sql """insert into emps values("2020-01-01",1,"a",1,1,1);""" @@ -52,5 +51,6 @@ suite ("testSubQuery") { qt_select_mv "select empid, deptno, salary from emps e1 where empid = (select max(empid) from emps where deptno = e1.deptno) order by deptno;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") } diff --git a/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy b/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy index f504c81f214e91..c6243a3f923616 100644 --- a/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy +++ b/regression-test/suites/mv_p0/ut/testUnionDistinct/testUnionDistinct.groovy @@ -35,6 +35,7 @@ suite ("testUnionDistinct") { sql """insert into emps values("2020-01-02",2,"b",2,2,2);""" sql """insert into emps values("2020-01-03",3,"c",3,3,3);""" + createMV("create materialized view emps_mv as select empid, deptno from emps order by empid, deptno;") sql """insert into emps values("2020-01-01",1,"a",1,1,1);""" @@ -52,6 +53,7 @@ suite ("testUnionDistinct") { } qt_select_mv "select * from (select empid, deptno from emps where empid >1 union select empid, deptno from emps where empid <0) t order by 1;" sql """set enable_stats=true;""" + sql """alter table emps modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from emps order by empid;", "emps_mv") explain { diff --git a/regression-test/suites/nereids_function_p0/agg_function/test_orthogonal_bitmap_expr_calculate.groovy b/regression-test/suites/nereids_function_p0/agg_function/test_orthogonal_bitmap_expr_calculate.groovy new file mode 100644 index 00000000000000..b47315e96dbcec --- /dev/null +++ b/regression-test/suites/nereids_function_p0/agg_function/test_orthogonal_bitmap_expr_calculate.groovy @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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_orthogonal_bitmap_expr_calculate") { + multi_sql """ + drop table if exists test_orthogonal_bitmap_expr_calculate; + + create table test_orthogonal_bitmap_expr_calculate( + id int, + tag int, + user_id bitmap bitmap_union + ) + aggregate key(id, tag) + distributed by hash(id) buckets 1 + properties( + 'replication_num'='1' + ); + + insert into test_orthogonal_bitmap_expr_calculate values + (1, 100, bitmap_from_string('1,2,3,4,5')), + (1, 200, bitmap_from_string('3,4,5,6,7')); + + set enable_fallback_to_original_planner=false; + """ + + test { + sql """ + select bitmap_to_string(orthogonal_bitmap_expr_calculate(user_id, tag, '(100&200)')) + from test_orthogonal_bitmap_expr_calculate + """ + result([['3,4,5']]) + } + + test { + sql """ + select orthogonal_bitmap_expr_calculate_count(user_id, tag, '(100&200)') + from test_orthogonal_bitmap_expr_calculate + """ + result([[3L]]) + } +} diff --git a/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_filter.groovy b/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_filter.groovy new file mode 100644 index 00000000000000..4d2b17e1c853e9 --- /dev/null +++ b/regression-test/suites/nereids_p0/sql_functions/array_functions/test_array_filter.groovy @@ -0,0 +1,204 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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_array_filter") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + // array functions only supported in vectorized engine + sql """DROP TABLE IF EXISTS table_30_un_pa_ke_pr_di4 """ + sql """ CREATE TABLE IF NOT EXISTS table_30_un_pa_ke_pr_di4 ( + `pk` int NULL, + `col_int_undef_signed_index_inverted` int NULL, + `col_int_undef_signed` int NULL, + `col_int_undef_signed_not_null` int NOT NULL, + `col_int_undef_signed_not_null_index_inverted` int NOT NULL, + `col_date_undef_signed` date NULL, + `col_date_undef_signed_index_inverted` date NULL, + `col_date_undef_signed_not_null` date NOT NULL, + `col_date_undef_signed_not_null_index_inverted` date NOT NULL, + `col_varchar_1024__undef_signed` varchar(1024) NULL, + `col_varchar_1024__undef_signed_index_inverted` varchar(1024) NULL, + `col_varchar_1024__undef_signed_index_inverted_p_e` varchar(1024) NULL, + `col_varchar_1024__undef_signed_index_inverted_p_u` varchar(1024) NULL, + `col_varchar_1024__undef_signed_not_null` varchar(1024) NOT NULL, + `col_varchar_1024__undef_signed_not_null_index_inverted` varchar(1024) NOT NULL, + `col_varchar_1024__undef_signed_not_null_index_inverted_p_e` varchar(1024) NOT NULL, + `col_varchar_1024__undef_signed_not_null_index_inverted_p_u` varchar(1024) NOT NULL, + `col_boolean_undef_signed` boolean NULL, + `col_boolean_undef_signed_not_null` boolean NOT NULL, + `col_decimal_18__6__undef_signed` decimal(18,6) NULL, + `col_decimal_18__6__undef_signed_index_inverted` decimal(18,6) NULL, + `col_decimal_18__6__undef_signed_not_null` decimal(18,6) NOT NULL, + `col_decimal_18__6__undef_signed_not_null_index_inverted` decimal(18,6) NOT NULL, + `col_datetime_3__undef_signed` datetime(3) NULL, + `col_datetime_3__undef_signed_index_inverted` datetime(3) NULL, + `col_datetime_3__undef_signed_not_null` datetime(3) NOT NULL, + `col_datetime_3__undef_signed_not_null_index_inverted` datetime(3) NOT NULL, + `col_datetime_6__undef_signed` datetime(6) NULL, + `col_datetime_6__undef_signed_index_inverted` datetime(6) NULL, + `col_datetime_6__undef_signed_not_null` datetime(6) NOT NULL, + `col_datetime_6__undef_signed_not_null_index_inverted` datetime(6) NOT NULL, + `col_array_boolean__undef_signed` array NULL, + `col_array_boolean__undef_signed_index_inverted` array NULL, + `col_array_boolean__undef_signed_not_null` array NOT NULL, + `col_array_boolean__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_tinyint__undef_signed` array NULL, + `col_array_tinyint__undef_signed_index_inverted` array NULL, + `col_array_tinyint__undef_signed_not_null` array NOT NULL, + `col_array_tinyint__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_smallint__undef_signed` array NULL, + `col_array_smallint__undef_signed_index_inverted` array NULL, + `col_array_smallint__undef_signed_not_null` array NOT NULL, + `col_array_smallint__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_int__undef_signed` array NULL, + `col_array_int__undef_signed_index_inverted` array NULL, + `col_array_int__undef_signed_not_null` array NOT NULL, + `col_array_int__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_bigint__undef_signed` array NULL, + `col_array_bigint__undef_signed_index_inverted` array NULL, + `col_array_bigint__undef_signed_not_null` array NOT NULL, + `col_array_bigint__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_largeint__undef_signed` array NULL, + `col_array_largeint__undef_signed_index_inverted` array NULL, + `col_array_largeint__undef_signed_not_null` array NOT NULL, + `col_array_largeint__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_decimal_37_12___undef_signed` array NULL, + `col_array_decimal_37_12___undef_signed_index_inverted` array NULL, + `col_array_decimal_37_12___undef_signed_not_null` array NOT NULL, + `col_array_decimal_37_12___undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_decimal_8_4___undef_signed` array NULL, + `col_array_decimal_8_4___undef_signed_index_inverted` array NULL, + `col_array_decimal_8_4___undef_signed_not_null` array NOT NULL, + `col_array_decimal_8_4___undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_decimal_9_0___undef_signed` array NULL, + `col_array_decimal_9_0___undef_signed_index_inverted` array NULL, + `col_array_decimal_9_0___undef_signed_not_null` array NOT NULL, + `col_array_decimal_9_0___undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_char_255___undef_signed` array NULL, + `col_array_char_255___undef_signed_index_inverted` array NULL, + `col_array_char_255___undef_signed_not_null` array NOT NULL, + `col_array_char_255___undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_varchar_65533___undef_signed` array NULL, + `col_array_varchar_65533___undef_signed_index_inverted` array NULL, + `col_array_varchar_65533___undef_signed_not_null` array NOT NULL, + `col_array_varchar_65533___undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_string__undef_signed` array NULL, + `col_array_string__undef_signed_index_inverted` array NULL, + `col_array_string__undef_signed_not_null` array NOT NULL, + `col_array_string__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_date__undef_signed` array NULL, + `col_array_date__undef_signed_index_inverted` array NULL, + `col_array_date__undef_signed_not_null` array NOT NULL, + `col_array_date__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_datetime__undef_signed` array NULL, + `col_array_datetime__undef_signed_index_inverted` array NULL, + `col_array_datetime__undef_signed_not_null` array NOT NULL, + `col_array_datetime__undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_datetime_3___undef_signed` array NULL, + `col_array_datetime_3___undef_signed_index_inverted` array NULL, + `col_array_datetime_3___undef_signed_not_null` array NOT NULL, + `col_array_datetime_3___undef_signed_not_null_index_inverted` array NOT NULL, + `col_array_datetime_6___undef_signed` array NULL, + `col_array_datetime_6___undef_signed_index_inverted` array NULL, + `col_array_datetime_6___undef_signed_not_null` array NOT NULL, + `col_array_datetime_6___undef_signed_not_null_index_inverted` array NOT NULL, + INDEX col_int_undef_signed_index_inverted_idx (`col_int_undef_signed_index_inverted`) USING INVERTED, + INDEX col_int_undef_signed_not_null_index_inverted_idx (`col_int_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_index_inverted_idx (`col_date_undef_signed_index_inverted`) USING INVERTED, + INDEX col_date_undef_signed_not_null_index_inverted_idx (`col_date_undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inverted_idx (`col_varchar_1024__undef_signed_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_index_inverted_p_e_idx (`col_varchar_1024__undef_signed_index_inverted_p_e`) USING INVERTED PROPERTIES("parser" = "english", "lower_case" = "true", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_index_inverted_p_u_idx (`col_varchar_1024__undef_signed_index_inverted_p_u`) USING INVERTED PROPERTIES("parser" = "unicode", "lower_case" = "true", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_not_null_index_inverted_idx (`col_varchar_1024__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_varchar_1024__undef_signed_not_null_index_inverted_p_e_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_p_e`) USING INVERTED PROPERTIES("parser" = "english", "lower_case" = "true", "support_phrase" = "true"), + INDEX col_varchar_1024__undef_signed_not_null_index_inverted_p_u_idx (`col_varchar_1024__undef_signed_not_null_index_inverted_p_u`) USING INVERTED PROPERTIES("parser" = "unicode", "lower_case" = "true", "support_phrase" = "true"), + INDEX col_decimal_18__6__undef_signed_index_inverted_idx (`col_decimal_18__6__undef_signed_index_inverted`) USING INVERTED, + INDEX col_decimal_18__6__undef_signed_not_null_index_inverted_idx (`col_decimal_18__6__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_datetime_3__undef_signed_index_inverted_idx (`col_datetime_3__undef_signed_index_inverted`) USING INVERTED, + INDEX col_datetime_3__undef_signed_not_null_index_inverted_idx (`col_datetime_3__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_datetime_6__undef_signed_index_inverted_idx (`col_datetime_6__undef_signed_index_inverted`) USING INVERTED, + INDEX col_datetime_6__undef_signed_not_null_index_inverted_idx (`col_datetime_6__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_boolean__undef_signed_index_inverted_idx (`col_array_boolean__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_boolean__undef_signed_not_null_index_inverted_idx (`col_array_boolean__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_tinyint__undef_signed_index_inverted_idx (`col_array_tinyint__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_tinyint__undef_signed_not_null_index_inverted_idx (`col_array_tinyint__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_smallint__undef_signed_index_inverted_idx (`col_array_smallint__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_smallint__undef_signed_not_null_index_inverted_idx (`col_array_smallint__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_int__undef_signed_index_inverted_idx (`col_array_int__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_int__undef_signed_not_null_index_inverted_idx (`col_array_int__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_bigint__undef_signed_index_inverted_idx (`col_array_bigint__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_bigint__undef_signed_not_null_index_inverted_idx (`col_array_bigint__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_largeint__undef_signed_index_inverted_idx (`col_array_largeint__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_largeint__undef_signed_not_null_index_inverted_idx (`col_array_largeint__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_decimal_37_12___undef_signed_index_inverted_idx (`col_array_decimal_37_12___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_decimal_37_12___undef_signed_n_0_idx (`col_array_decimal_37_12___undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_decimal_8_4___undef_signed_index_inverted_idx (`col_array_decimal_8_4___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_decimal_8_4___undef_signed_not_1_idx (`col_array_decimal_8_4___undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_decimal_9_0___undef_signed_index_inverted_idx (`col_array_decimal_9_0___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_decimal_9_0___undef_signed_not_2_idx (`col_array_decimal_9_0___undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_char_255___undef_signed_index_inverted_idx (`col_array_char_255___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_char_255___undef_signed_not_null_index_inverted_idx (`col_array_char_255___undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_varchar_65533___undef_signed_index_inverted_idx (`col_array_varchar_65533___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_varchar_65533___undef_signed_n_3_idx (`col_array_varchar_65533___undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_string__undef_signed_index_inverted_idx (`col_array_string__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_string__undef_signed_not_null_index_inverted_idx (`col_array_string__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_date__undef_signed_index_inverted_idx (`col_array_date__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_date__undef_signed_not_null_index_inverted_idx (`col_array_date__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_datetime__undef_signed_index_inverted_idx (`col_array_datetime__undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_datetime__undef_signed_not_null_index_inverted_idx (`col_array_datetime__undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_datetime_3___undef_signed_index_inverted_idx (`col_array_datetime_3___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_datetime_3___undef_signed_not_null_index_inverted_idx (`col_array_datetime_3___undef_signed_not_null_index_inverted`) USING INVERTED, + INDEX col_array_datetime_6___undef_signed_index_inverted_idx (`col_array_datetime_6___undef_signed_index_inverted`) USING INVERTED, + INDEX col_array_datetime_6___undef_signed_not_null_index_inverted_idx (`col_array_datetime_6___undef_signed_not_null_index_inverted`) USING INVERTED + ) ENGINE=OLAP + DUPLICATE KEY(`pk`, `col_int_undef_signed_index_inverted`) + DISTRIBUTED BY HASH(`pk`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "bloom_filter_columns" = "col_varchar_1024__undef_signed, col_date_undef_signed_not_null, col_date_undef_signed, col_int_undef_signed, col_varchar_1024__undef_signed_not_null, col_int_undef_signed_not_null", + "storage_medium" = "hdd", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false" + ); """ + + streamLoad { + table 'table_30_un_pa_ke_pr_di4' + set 'column_separator', '\t' + file 'array_filter.csv' + time 60000 + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals(236, json.NumberTotalRows) + assertEquals(236, json.NumberLoadedRows) + } + } + + sql """ delete from table_30_un_pa_ke_pr_di4 where col_varchar_1024__undef_signed_index_inverted_p_u = '始终'; + delete from table_30_un_pa_ke_pr_di4 where col_varchar_1024__undef_signed = ''; + delete from table_30_un_pa_ke_pr_di4 where col_varchar_1024__undef_signed_not_null = '--'; + delete from table_30_un_pa_ke_pr_di4 where col_int_undef_signed = -1; + delete from table_30_un_pa_ke_pr_di4 where col_varchar_1024__undef_signed_not_null_index_inverted_p_u = '刘伟'; """ + order_qt_sql """ + SELECT array_contains(col_array_string__undef_signed_not_null, col_varchar_1024__undef_signed_index_inverted_p_e), (arrays_overlap(col_array_date__undef_signed_index_inverted, col_array_date__undef_signed_not_null)) and not (arrays_overlap(col_array_smallint__undef_signed_not_null, col_array_boolean__undef_signed_not_null) or (array_contains(col_array_decimal_8_4___undef_signed_index_inverted, col_int_undef_signed)) and (arrays_overlap(col_array_bigint__undef_signed, [ 32767, -9223372036854775808 ])) and ((case col_boolean_undef_signed when 0 then 1 when 1 then 2 else 0 end) = 0)) FROM table_30_un_pa_ke_pr_di4 where array_contains(col_array_largeint__undef_signed_not_null_index_inverted, col_boolean_undef_signed) or ((arrays_overlap(col_array_tinyint__undef_signed_not_null, col_array_largeint__undef_signed_not_null_index_inverted)) and (array_contains(col_array_date__undef_signed_not_null, col_date_undef_signed_not_null)) and not ((array_contains(col_array_largeint__undef_signed, col_int_undef_signed_index_inverted)) and not ((col_array_largeint__undef_signed_index_inverted is null) and col_array_decimal_8_4___undef_signed_not_null_index_inverted is not null or not (arrays_overlap(col_array_int__undef_signed_not_null, col_array_int__undef_signed_not_null) and not (case col_date_undef_signed_not_null_index_inverted when "2027-01-09" then 1 when "2023-01-15 08:32:59.123123" then 2 else 0 end) = 1)))) order by pk,col_int_undef_signed_index_inverted; """ + +} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy new file mode 100644 index 00000000000000..9134d66b76d751 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy @@ -0,0 +1,125 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("push_down_aggr_distinct_through_join_one_side_cust") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "set DISABLE_NEREIDS_RULES='PRUNE_EMPTY_PARTITION, ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" + + sql """ + DROP TABLE IF EXISTS dwd_com_abtest_result_inc_ymds; + DROP TABLE IF EXISTS dwd_tracking_sensor_init_tmp_ymds; + """ + + sql """ + CREATE TABLE `dwd_com_abtest_result_inc_ymds` ( + `app_name` varchar(255) NULL, + `user_key` text NULL, + `group_name` text NULL, + `dt` date NOT NULL, + ) ENGINE=OLAP + DUPLICATE KEY(`app_name`) + AUTO PARTITION BY RANGE (date_trunc(`dt`, 'day')) + (PARTITION p20240813000000 VALUES [('2024-08-13'), ('2024-08-14')), + PARTITION p20240814000000 VALUES [('2024-08-14'), ('2024-08-15')), + PARTITION p20240815000000 VALUES [('2024-08-15'), ('2024-08-16')), + PARTITION p20240816000000 VALUES [('2024-08-16'), ('2024-08-17')), + PARTITION p20240817000000 VALUES [('2024-08-17'), ('2024-08-18')), + PARTITION p20240818000000 VALUES [('2024-08-18'), ('2024-08-19')), + PARTITION p20240819000000 VALUES [('2024-08-19'), ('2024-08-20'))) + DISTRIBUTED BY HASH(`app_name`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728" + ); + + CREATE TABLE `dwd_tracking_sensor_init_tmp_ymds` ( + `ip` varchar(20) NULL, + `gz_user_id` text NULL, + `dt` date NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`ip`) + AUTO PARTITION BY RANGE (date_trunc(`dt`, 'day')) + (PARTITION p20240813000000 VALUES [('2024-08-13'), ('2024-08-14')), + PARTITION p20240814000000 VALUES [('2024-08-14'), ('2024-08-15')), + PARTITION p20240815000000 VALUES [('2024-08-15'), ('2024-08-16')), + PARTITION p20240816000000 VALUES [('2024-08-16'), ('2024-08-17')), + PARTITION p20240817000000 VALUES [('2024-08-17'), ('2024-08-18')), + PARTITION p20240818000000 VALUES [('2024-08-18'), ('2024-08-19')), + PARTITION p20240819000000 VALUES [('2024-08-19'), ('2024-08-20'))) + DISTRIBUTED BY HASH(`ip`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728" + ); + """ + + explain { + sql("physical PLAN SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/" + + "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymds.gz_user_id) AS a2c1a830_1," + + "dwd_com_abtest_result_inc_ymds.group_name AS ab1011d6," + + "dwd_tracking_sensor_init_tmp_ymds.dt AS ad466123 " + + "FROM dwd_tracking_sensor_init_tmp_ymds " + + "LEFT JOIN dwd_com_abtest_result_inc_ymds " + + "ON dwd_tracking_sensor_init_tmp_ymds.gz_user_id = dwd_com_abtest_result_inc_ymds.user_key " + + "AND dwd_tracking_sensor_init_tmp_ymds.dt = dwd_com_abtest_result_inc_ymds.dt " + + "WHERE dwd_tracking_sensor_init_tmp_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "AND dwd_com_abtest_result_inc_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "GROUP BY 2, 3 ORDER BY 3 asc limit 10000;"); + contains"groupByExpr=[gz_user_id#1, dt#2]" + contains"groupByExpr=[gz_user_id#1, dt#2, group_name#5], outputExpr=[gz_user_id#1, dt#2, group_name#5]" + contains"[group_name#5, dt#2]" + contains"groupByExpr=[group_name#5, dt#2], outputExpr=[group_name#5, dt#2, count(partial_count(gz_user_id)#12) AS `a2c1a830_1`#7]" + } + + explain { + sql("physical PLAN SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/" + + "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymds.ip) AS a2c1a830_1," + + "dwd_com_abtest_result_inc_ymds.group_name AS ab1011d6," + + "dwd_tracking_sensor_init_tmp_ymds.dt AS ad466123 " + + "FROM dwd_tracking_sensor_init_tmp_ymds " + + "LEFT JOIN dwd_com_abtest_result_inc_ymds " + + "ON dwd_tracking_sensor_init_tmp_ymds.gz_user_id = dwd_com_abtest_result_inc_ymds.user_key " + + "AND dwd_tracking_sensor_init_tmp_ymds.dt = dwd_com_abtest_result_inc_ymds.dt " + + "WHERE dwd_tracking_sensor_init_tmp_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "AND dwd_com_abtest_result_inc_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "GROUP BY 2, 3 ORDER BY 3 asc limit 10000;"); + contains"groupByExpr=[ip#0, gz_user_id#1, dt#2], outputExpr=[ip#0, gz_user_id#1, dt#2]" + contains"groupByExpr=[ip#0, dt#2, group_name#5], outputExpr=[ip#0, dt#2, group_name#5]" + contains"groupByExpr=[group_name#5, dt#2], outputExpr=[group_name#5, dt#2, partial_count(ip#0) AS `partial_count(ip)`#12]" + contains"groupByExpr=[group_name#5, dt#2], outputExpr=[group_name#5, dt#2, count(partial_count(ip)#12) AS `a2c1a830_1`#7]" + } +} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..65a6b15ceccf73 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -0,0 +1,257 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("push_down_count_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql """ + DROP TABLE IF EXISTS count_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS count_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into count_with_distinct_t values (1, 1, 'a')" + sql "insert into count_with_distinct_t values (2, null, 'a')" + sql "insert into count_with_distinct_t values (3, 1, null)" + sql "insert into count_with_distinct_t values (4, 2, 'b')" + sql "insert into count_with_distinct_t values (5, null, 'b')" + sql "insert into count_with_distinct_t values (6, 2, null)" + sql "insert into count_with_distinct_t values (7, 3, 'c')" + sql "insert into count_with_distinct_t values (8, null, 'c')" + sql "insert into count_with_distinct_t values (9, 3, null)" + sql "insert into count_with_distinct_t values (10, null, null)" + sql "analyze table count_with_distinct_t with full with sync;" + + order_qt_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 right join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 full join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 inner join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left anti join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t) count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id join count_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from count_with_distinct_t t1_alias join count_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 right join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 full join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 inner join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left anti join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t) count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id join count_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from count_with_distinct_t t1_alias join count_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index 02e06710296333..e551fa04c9110a 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -426,4 +426,99 @@ suite("push_down_count_through_join_one_side") { qt_with_hint_groupby_pushdown_nested_queries """ explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from (select * from count_t_one_side where score > 20) t1 join (select * from count_t_one_side where id < 100) t2 on t1.id = t2.id group by t1.name; """ + + sql """ + drop table if exists dw_user_b2c_tracking_info_tmp_ymd; + create table dw_user_b2c_tracking_info_tmp_ymd ( + guid int, + dt varchar, + first_visit_time varchar + )Engine=Olap + DUPLICATE KEY(guid) + distributed by hash(dt) buckets 3 + properties('replication_num' = '1'); + + insert into dw_user_b2c_tracking_info_tmp_ymd values (1, '2024-08-19', '2024-08-19'); + + drop table if exists dwd_tracking_sensor_init_tmp_ymd; + create table dwd_tracking_sensor_init_tmp_ymd ( + guid int, + dt varchar, + tracking_type varchar + )Engine=Olap + DUPLICATE KEY(guid) + distributed by hash(dt) buckets 3 + properties('replication_num' = '1'); + + insert into dwd_tracking_sensor_init_tmp_ymd values(1, '2024-08-19', 'click'), (1, '2024-08-19', 'click'); + """ + sql """ + set ENABLE_NEREIDS_RULES = "PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE"; + set disable_join_reorder=true; + """ + + qt_shape """ + explain shape plan + SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE)*/ + Count(*) AS accee593, + CASE + WHEN dwd_tracking_sensor_init_tmp_ymd.dt = + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '是' + WHEN dwd_tracking_sensor_init_tmp_ymd.dt > + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '否' + ELSE '-1' + end AS a1302fb2, + dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 + FROM dwd_tracking_sensor_init_tmp_ymd + LEFT JOIN dw_user_b2c_tracking_info_tmp_ymd + ON dwd_tracking_sensor_init_tmp_ymd.guid = + dw_user_b2c_tracking_info_tmp_ymd.guid + AND dwd_tracking_sensor_init_tmp_ymd.dt = + dw_user_b2c_tracking_info_tmp_ymd.dt + WHERE dwd_tracking_sensor_init_tmp_ymd.dt = '2024-08-19' + AND dw_user_b2c_tracking_info_tmp_ymd.dt = '2024-08-19' + AND dwd_tracking_sensor_init_tmp_ymd.dt >= + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, 10) + AND dwd_tracking_sensor_init_tmp_ymd.tracking_type = 'click' + GROUP BY 2, + 3 + ORDER BY 3 ASC + LIMIT 10000; + """ + + qt_agg_pushed """ + SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE)*/ + Count(*) AS accee593, + CASE + WHEN dwd_tracking_sensor_init_tmp_ymd.dt = + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '是' + WHEN dwd_tracking_sensor_init_tmp_ymd.dt > + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, + 10) THEN + '否' + ELSE '-1' + end AS a1302fb2, + dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 + FROM dwd_tracking_sensor_init_tmp_ymd + LEFT JOIN dw_user_b2c_tracking_info_tmp_ymd + ON dwd_tracking_sensor_init_tmp_ymd.guid = + dw_user_b2c_tracking_info_tmp_ymd.guid + AND dwd_tracking_sensor_init_tmp_ymd.dt = + dw_user_b2c_tracking_info_tmp_ymd.dt + WHERE dwd_tracking_sensor_init_tmp_ymd.dt = '2024-08-19' + AND dw_user_b2c_tracking_info_tmp_ymd.dt = '2024-08-19' + AND dwd_tracking_sensor_init_tmp_ymd.dt >= + Substring(dw_user_b2c_tracking_info_tmp_ymd.first_visit_time, 1, 10) + AND dwd_tracking_sensor_init_tmp_ymd.tracking_type = 'click' + GROUP BY 2, + 3 + ORDER BY 3 ASC + LIMIT 10000; + """ } diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..676becbcb6ecfa --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy @@ -0,0 +1,256 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("push_down_min_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql """ + DROP TABLE IF EXISTS min_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS min_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into min_with_distinct_t values (1, 1, 'a')" + sql "insert into min_with_distinct_t values (2, null, 'a')" + sql "insert into min_with_distinct_t values (3, 1, null)" + sql "insert into min_with_distinct_t values (4, 2, 'b')" + sql "insert into min_with_distinct_t values (5, null, 'b')" + sql "insert into min_with_distinct_t values (6, 2, null)" + sql "insert into min_with_distinct_t values (7, 3, 'c')" + sql "insert into min_with_distinct_t values (8, null, 'c')" + sql "insert into min_with_distinct_t values (9, 3, null)" + sql "insert into min_with_distinct_t values (10, null, null)" + sql "analyze table min_with_distinct_t with sync;" + order_qt_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 right join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 full join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 inner join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left anti join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t) min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id join min_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from min_with_distinct_t t1_alias join min_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 right join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 full join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 inner join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left anti join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t) min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id join min_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from min_with_distinct_t t1_alias join min_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..799d45ba413fb7 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy @@ -0,0 +1,252 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("push_down_sum_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql """ + DROP TABLE IF EXISTS sum_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS sum_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into sum_with_distinct_t values (1, 1, 'a')" + sql "insert into sum_with_distinct_t values (2, null, 'a')" + sql "insert into sum_with_distinct_t values (3, 1, null)" + sql "insert into sum_with_distinct_t values (4, 2, 'b')" + sql "insert into sum_with_distinct_t values (5, null, 'b')" + sql "insert into sum_with_distinct_t values (6, 2, null)" + sql "insert into sum_with_distinct_t values (7, 3, 'c')" + sql "insert into sum_with_distinct_t values (8, null, 'c')" + sql "insert into sum_with_distinct_t values (9, 3, null)" + sql "insert into sum_with_distinct_t values (10, null, null)" + sql "analyze table sum_with_distinct_t with sync;" + order_qt_groupby_pushdown_basic """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 right join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 full join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 inner join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 left anti join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select sum(distinct t1.score), avg(t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select sum(distinct t1.score) from (select * from (select * from sum_with_distinct_t) sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select sum(distinct t1.score), sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id join sum_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select t1.name, sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select t1.name, sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select sum(distinct t1_alias.score) from sum_with_distinct_t t1_alias join sum_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 20) t1 join (select * from sum_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 right join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 full join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 inner join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 left anti join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select sum(distinct t1.score), avg(t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select sum(distinct t1.score) from (select * from (select * from sum_with_distinct_t) sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select sum(distinct t1.score), sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id join sum_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select t1.name, sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select t1.name, sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select sum(distinct t1_alias.score) from sum_with_distinct_t t1_alias join sum_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 20) t1 join (select * from sum_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy new file mode 100644 index 00000000000000..89abe58fc1f003 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eliminate_gby_key/eliminate_group_by_key_by_uniform.groovy @@ -0,0 +1,221 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +suite("eliminate_group_by_key_by_uniform") { + sql "set enable_nereids_rules = 'ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" + sql "drop table if exists eli_gbk_by_uniform_t" + sql """create table eli_gbk_by_uniform_t(a int null, b int not null, c varchar(10) null, d date, dt datetime) + distributed by hash(a) properties("replication_num"="1"); + """ + qt_empty_tranform_not_to_scalar_agg "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 group by a" + qt_empty_tranform_multi_column "select a,b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=2 group by a,b" + + sql """ + INSERT INTO eli_gbk_by_uniform_t (a, b, c, d, dt) VALUES + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00'), + (1, 100, 'apple', '2023-01-01', '2023-01-01 10:00:00'), + (2, 101, 'banana', '2023-01-02', '2023-01-02 11:00:00'), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00'), + (3, 102, 'cherry', '2023-01-03', '2023-01-03 12:00:00'), + (NULL, 103, 'date', '2023-01-04', '2023-01-04 13:00:00'), + (4, 104, 'elderberry', '2023-01-05', '2023-01-05 14:00:00'), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00'), + (5, 105, NULL, '2023-01-06', '2023-01-06 15:00:00'), + (6, 106, 'fig', '2023-01-07', '2023-01-07 16:00:00'), + (NULL, 107, 'grape', '2023-01-08', '2023-01-08 17:00:00'); + """ + qt_empty_tranform_multi_column "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 group by a, b,'abc' order by 1,2,3,4" + qt_tranform_to_scalar_agg_not_null_column "select b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where b = 1 group by a, b order by 1,2,3,4" + + qt_project_const "select sum(c1), c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1,2;" + qt_project_slot_uniform "select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t group by c1,c2,c3 order by 1,2,3,4;" + + qt_upper_refer "select b from (select b, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where b = 1 group by a, b) t order by b" + qt_upper_refer_varchar_alias "select c1,c2 from (select c as c1, min(a) c2, sum(a), count(a) from eli_gbk_by_uniform_t where c = 'cherry' group by a, b,c) t order by c1,c2" + qt_upper_refer_date "select d from (select d, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where d = '2023-01-06' group by d,a) t order by 1" + qt_upper_refer_datetime_not_to_scalar_agg "select dt from (select dt, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where dt = '2023-01-06 15:00:00' group by dt) t order by 1" + qt_upper_refer_datetime "select dt from (select dt, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where dt = '2023-01-06 15:00:00' group by dt, a) t order by 1" + + qt_project_no_other_agg_func "select c2 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;" + qt_project_const_not_to_scalar_agg_multi "select c2 from (select a c1,1 c2, 3 c3 from eli_gbk_by_uniform_t) t group by c2,c3 order by 1;" + qt_not_to_scalar_agg_multi "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and b=100 group by a, b,'abc' order by 1,2,3,4" + qt_conflict_equal_value "select a, min(a), sum(a), count(a) from eli_gbk_by_uniform_t where a = 1 and a=2 group by a, b,'abc' order by 1,2,3,4" + qt_project_slot_uniform_confict_value "select max(c3), c1,c2,c3 from (select a c1,1 c2, d c3 from eli_gbk_by_uniform_t where a=1) t where c2=2 group by c1,c2,c3 order by 1,2,3,4;" + + // test join + qt_inner_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_inner_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 inner join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 left join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_right_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_right_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 right join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_left_semi_join_right_has_filter "select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a order by 1" + qt_left_semi_join_left_has_filter "select t1.b from eli_gbk_by_uniform_t t1 left semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_right_has_on_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_left_has_on_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t1.b,t1.a order by 1" + qt_left_anti_join_left_has_where_filter "select t1.b from eli_gbk_by_uniform_t t1 left anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t1.b=100 group by t1.b,t1.a order by 1" + qt_right_semi_join_right_has_filter "select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.c order by 1" + qt_right_semi_join_left_has_filter "select t2.b from eli_gbk_by_uniform_t t1 right semi join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_right_has_on_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t2.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_left_has_on_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b and t1.b=100 group by t2.b,t2.c order by 1" + qt_right_anti_join_right_has_where_filter "select t2.b from eli_gbk_by_uniform_t t1 right anti join eli_gbk_by_uniform_t t2 on t1.b=t2.b where t2.b=100 group by t2.b,t2.c order by 1" + qt_cross_join_left_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 cross join eli_gbk_by_uniform_t t2 where t1.b=100 group by t1.b,t2.b,t2.c order by 1,2" + qt_cross_join_right_has_filter "select t1.b,t2.b from eli_gbk_by_uniform_t t1 cross join eli_gbk_by_uniform_t t2 where t2.b=100 group by t1.b,t2.b,t2.c order by 1,2" + + //test union + qt_union "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union select a,b from eli_gbk_by_uniform_t where b=100 group by a,b union select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_union_all "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union all select a,b from eli_gbk_by_uniform_t where b=100 group by a,b union all select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_intersect "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b intersect select a,b from eli_gbk_by_uniform_t where b=100 group by a,b intersect select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_except "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b except select a,b from eli_gbk_by_uniform_t where b=100 group by a,b except select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + qt_set_op_mixed "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b union select a,b from eli_gbk_by_uniform_t where b=100 group by a,b except select a,b from eli_gbk_by_uniform_t where a=5 group by a,b) t order by 1,2,3,4,5" + + //test window + qt_window "select max(a) over(partition by a order by a) from eli_gbk_by_uniform_t where a=10 group by a,b order by 1" + //test partition topn + qt_partition_topn "select r from (select rank() over(partition by a order by a) r from eli_gbk_by_uniform_t where a=10 group by a,b) t where r<2 order by 1" + + //test cte + qt_cte_producer "with t as (select a,b,count(*) from eli_gbk_by_uniform_t where a=1 group by a,b) select t1.a,t2.a,t2.b from t t1 inner join t t2 on t1.a=t2.a order by 1,2,3" + qt_cte_multi_producer "with t as (select a,b,count(*) from eli_gbk_by_uniform_t where a=1 group by a,b), tt as (select a,b,count(*) from eli_gbk_by_uniform_t where b=10 group by a,b) select t1.a,t2.a,t2.b from t t1 inner join tt t2 on t1.a=t2.a order by 1,2,3" + qt_cte_consumer "with t as (select * from eli_gbk_by_uniform_t) select t1.a,t2.b from t t1 inner join t t2 on t1.a=t2.a where t1.a=10 group by t1.a,t2.b order by 1,2 " + + //test filter + qt_filter "select * from (select a,b from eli_gbk_by_uniform_t where a=1 group by a,b) t where a>0 order by 1,2" + + //test topn + qt_topn "select a,b from eli_gbk_by_uniform_t where a=1 group by a,b order by a limit 10 offset 0" + + //olap table sink + sql "insert into eli_gbk_by_uniform_t select a,b,c,d,dt from eli_gbk_by_uniform_t where a = 1 group by a,b,c,d,dt" + qt_sink "select * from eli_gbk_by_uniform_t order by 1,2,3,4,5" + + sql """ + drop table if exists orders_inner_1 + """ + + sql """CREATE TABLE `orders_inner_1` ( + `o_orderkey` BIGINT not NULL, + `o_custkey` INT NULL, + `o_orderstatus` VARCHAR(1) NULL, + `o_totalprice` DECIMAL(15, 2) NULL, + `o_orderpriority` VARCHAR(15) NULL, + `o_clerk` VARCHAR(15) NULL, + `o_shippriority` INT NULL, + `o_comment` VARCHAR(79) NULL, + `o_orderdate` DATE NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + PARTITION BY list(o_orderkey) ( + PARTITION p1 VALUES in ('1'), + PARTITION p2 VALUES in ('2'), + PARTITION p3 VALUES in ('3'), + PARTITION p4 VALUES in ('4') + ) + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + drop table if exists lineitem_inner_1 + """ + + sql """CREATE TABLE `lineitem_inner_1` ( + `l_orderkey` BIGINT not NULL, + `l_linenumber` INT NULL, + `l_partkey` INT NULL, + `l_suppkey` INT NULL, + `l_quantity` DECIMAL(15, 2) NULL, + `l_extendedprice` DECIMAL(15, 2) NULL, + `l_discount` DECIMAL(15, 2) NULL, + `l_tax` DECIMAL(15, 2) NULL, + `l_returnflag` VARCHAR(1) NULL, + `l_linestatus` VARCHAR(1) NULL, + `l_commitdate` DATE NULL, + `l_receiptdate` DATE NULL, + `l_shipinstruct` VARCHAR(25) NULL, + `l_shipmode` VARCHAR(10) NULL, + `l_comment` VARCHAR(44) NULL, + `l_shipdate` DATE NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + PARTITION BY list(l_orderkey) ( + PARTITION p1 VALUES in ('1'), + PARTITION p2 VALUES in ('2'), + PARTITION p3 VALUES in ('3') + ) + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_inner_1 values + (2, 1, 'o', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (1, null, 'k', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'o', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'k', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'o', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); + """ + + sql """ + insert into lineitem_inner_1 values + (2, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, null, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); + """ + + qt_nest_exprid_replace """ + select l_shipdate, l_orderkey, t.O_ORDERDATE, t.o_orderkey, + count(t.O_ORDERDATE) over (partition by lineitem_inner_1.l_orderkey order by lineitem_inner_1.l_orderkey) as window_count + from lineitem_inner_1 + inner join (select O_ORDERDATE, o_orderkey, count(O_ORDERDATE) over (partition by O_ORDERDATE order by o_orderkey ) from orders_inner_1 where o_orderkey=2 group by O_ORDERDATE, o_orderkey) as t + on lineitem_inner_1.l_orderkey = t.o_orderkey + where t.o_orderkey=2 + group by l_shipdate, l_orderkey, t.O_ORDERDATE, t.o_orderkey + order by 1,2,3,4,5 + """ + sql "drop table if exists test1" + sql "drop table if exists test2" + sql "create table test1(a int, b int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into test1 values(1,1),(2,1),(3,1);" + sql "create table test2(a int, b int) distributed by hash(a) properties('replication_num'='1');" + sql "insert into test2 values(1,105),(2,105);" + qt_full_join_uniform_should_not_eliminate_group_by_key "select t2.b,t1.b from test1 t1 full join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_full2 "select t2.b,t1.b from (select * from test2 where b=105) t1 full join test1 t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + + qt_left_join_right_side_should_not_eliminate_group_by_key "select t2.b,t1.b from test1 t1 left join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_left_join_left_side_should_eliminate_group_by_key "select t2.b,t1.b from test1 t1 left join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t2.b,t1.b order by 1,2;" + + qt_right_join_left_side_should_not_eliminate_group_by_key "select t2.b,t1.b from (select * from test2 where b=105) t1 right join test1 t2 on t1.a=t2.a group by t2.b,t1.b order by 1,2;" + qt_right_join_right_side_should_eliminate_group_by_key "select t2.b,t1.b from (select * from test2 where b=105) t1 right join test1 t2 on t1.a=t2.a where t2.b=1 group by t2.b,t1.b order by 1,2;" + + qt_left_semi_left_side "select t1.b from test1 t1 left semi join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t1.b,t1.a order by 1;" + qt_left_anti_left_side "select t1.b from test1 t1 left anti join (select * from test2 where b=105) t2 on t1.a=t2.a where t1.b=1 group by t1.b,t1.a order by 1;" + qt_right_semi_right_side "select t2.b from test1 t1 right semi join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t2.a order by 1;" + qt_right_anti_right_side "select t2.b from test1 t1 right anti join (select * from test2 where b=105) t2 on t1.a=t2.a group by t2.b,t2.a order by 1;" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy index d10498705f60c6..db72f1174fdff2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_on_none_agg/agg_on_none_agg.groovy @@ -57,6 +57,8 @@ suite("agg_on_none_agg") { (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """ drop table if exists lineitem """ @@ -95,6 +97,8 @@ suite("agg_on_none_agg") { (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """ drop table if exists partsupp """ @@ -120,6 +124,8 @@ suite("agg_on_none_agg") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + sql """analyze table orders with sync;""" sql """analyze table lineitem with sync;""" sql """analyze table partsupp with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy index 355165055c8ef5..e0f33a2b0db8e5 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy @@ -120,6 +120,10 @@ suite("agg_optimize_when_uniform") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + sql """analyze table lineitem with sync;""" sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy index 3fa6dffca01367..2de47b4bbc2f8e 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_variety/agg_variety.groovy @@ -95,6 +95,8 @@ suite("agg_variety") { (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); """ + + sql """ drop table if exists partsupp """ @@ -124,6 +126,10 @@ suite("agg_variety") { sql """analyze table lineitem with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + def check_rewrite_but_not_chose = { mv_sql, query_sql, mv_name -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy index cda9b21fc4e51d..5b8aa6e00d2308 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy @@ -132,6 +132,9 @@ suite("aggregate_with_roll_up") { sql """analyze table partsupp with sync""" sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" // multi table // filter inside + left + use roll up dimension diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index 9d60280503cf74..356b96267a88f5 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -36,7 +36,8 @@ suite("aggregate_without_roll_up") { o_orderpriority CHAR(15) NOT NULL, o_clerk CHAR(15) NOT NULL, o_shippriority INTEGER NOT NULL, - o_comment VARCHAR(79) NOT NULL + o_comment VARCHAR(79) NOT NULL, + public_col INT NULL ) DUPLICATE KEY(o_orderkey, o_custkey) DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 @@ -44,11 +45,9 @@ suite("aggregate_without_roll_up") { "replication_num" = "1" ); """ - sql """ drop table if exists lineitem """ - sql""" CREATE TABLE IF NOT EXISTS lineitem ( l_orderkey INTEGER NOT NULL, @@ -66,7 +65,8 @@ suite("aggregate_without_roll_up") { l_receiptdate DATE NOT NULL, l_shipinstruct CHAR(25) NOT NULL, l_shipmode CHAR(10) NOT NULL, - l_comment VARCHAR(44) NOT NULL + l_comment VARCHAR(44) NOT NULL, + public_col INT NULL ) DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 @@ -74,18 +74,17 @@ suite("aggregate_without_roll_up") { "replication_num" = "1" ) """ - sql """ drop table if exists partsupp """ - sql """ CREATE TABLE IF NOT EXISTS partsupp ( ps_partkey INTEGER NOT NULL, ps_suppkey INTEGER NOT NULL, ps_availqty INTEGER NOT NULL, ps_supplycost DECIMALV3(15,2) NOT NULL, - ps_comment VARCHAR(199) NOT NULL + ps_comment VARCHAR(199) NOT NULL, + public_col INT NULL ) DUPLICATE KEY(ps_partkey, ps_suppkey) DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 @@ -96,31 +95,37 @@ suite("aggregate_without_roll_up") { sql """ insert into lineitem values - (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', 1), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', null), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', 2), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', null), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx', 3); """ sql """ insert into orders values - (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), - (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), - (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), - (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), - (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), - (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy', 1), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy', null), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy', 2), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy', null), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy', 3), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm', null), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi', 4), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi', null); """ sql """ insert into partsupp values - (2, 3, 9, 10.01, 'supply1'), - (2, 3, 10, 11.01, 'supply2'); + (2, 3, 9, 10.01, 'supply1', 1), + (2, 3, 10, 11.01, 'supply2', null); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // single table // with filter def mv1_0 = """ @@ -367,7 +372,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -565,7 +570,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -655,7 +660,7 @@ suite("aggregate_without_roll_up") { "max(o_totalprice) as max_total, " + "min(o_totalprice) as min_total, " + "count(*) as count_all, " + - "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as distinct_count " + "from lineitem " + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + "group by " + @@ -1356,4 +1361,615 @@ suite("aggregate_without_roll_up") { """ async_mv_rewrite_fail(db, mv27_0, query27_0, "mv27_0") sql """ DROP MATERIALIZED VIEW IF EXISTS mv27_0""" + + + // query and mv has the same filter but position is different, should rewrite successfully + def mv28_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + l_orderkey is null + or l_orderkey <> 8 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + l_orderkey is null + or l_orderkey <> 8 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query28_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + l_orderkey is null + or l_orderkey <> 8 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + l_orderkey is null + or l_orderkey <> 8 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query28_0_before "${query28_0}" + async_mv_rewrite_success(db, mv28_0, query28_0, "mv28_0") + order_qt_query28_0_after "${query28_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv28_0""" + + + + // query and mv has the same filter but position is different, should rewrite successfully + def mv29_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query29_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query29_0_before "${query29_0}" + async_mv_rewrite_success(db, mv29_0, query29_0, "mv29_0") + order_qt_query29_0_after "${query29_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv29_0""" + + + // query and mv has the same filter but position is different, should rewrite successfully + // mv join condition has alias + def mv30_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey_alias, + orders.public_col as col1, + l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey_alias, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2, + sum( + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey as o_orderkey_alias, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey as l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on lineitem.l_orderkey_alias = orders.o_orderkey_alias + inner join ( + select + ps_partkey as ps_partkey_alias, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on partsupp.ps_partkey_alias = orders.o_orderkey_alias + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey_alias = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query30_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query30_0_before "${query30_0}" + async_mv_rewrite_success(db, mv30_0, query30_0, "mv30_0") + order_qt_query30_0_after "${query30_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv30_0""" + + + // query and mv has the same filter but position is different, should rewrite successfully + // query join condition has alias + def mv31_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query31_0 = """ +select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey_alias, + orders.public_col as col1, + l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey_alias, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2, + sum( + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey as o_orderkey_alias, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey as l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on lineitem.l_orderkey_alias = orders.o_orderkey_alias + inner join ( + select + ps_partkey as ps_partkey_alias, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on partsupp.ps_partkey_alias = orders.o_orderkey_alias + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey_alias = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query31_0_before "${query31_0}" + async_mv_rewrite_success(db, mv31_0, query31_0, "mv31_0") + order_qt_query31_0_after "${query31_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv31_0""" } diff --git a/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy b/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy index 5a6c75f6c59c6e..b9e671f2636b82 100644 --- a/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy @@ -49,6 +49,7 @@ suite("grace_period") { "replication_num" = "1" ); """ + sql """alter table orders_partition modify column o_comment set stats ('row_count'='3');""" sql """ drop table if exists lineitem_partition @@ -81,6 +82,7 @@ suite("grace_period") { "replication_num" = "1" ); """ + sql """alter table lineitem_partition modify column l_comment set stats ('row_count'='3');""" sql """ insert into orders_partition values diff --git a/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy b/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy index ae349c6514c469..4af1d778c002b2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/availability/materialized_view_switch.groovy @@ -124,6 +124,11 @@ suite("materialized_view_switch") { analyze table orders with sync; analyze table partsupp with sync; """ + + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + def mv_name = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey from lineitem diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy index c86f5d6a141f7f..a2fcb2eba15913 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_datetrunc_part_up.groovy @@ -71,6 +71,7 @@ suite("mtmv_range_date_datetrunc_date_part_up") { (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-29 03:00:00'), (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-29 04:00:00'); """ + sql """alter table ${tb_name} modify column l_comment set stats ('row_count'='5');""" def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy index 5be6b1afdcdc44..4c5b01ff59fd3a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up.groovy @@ -119,6 +119,8 @@ suite("mtmv_range_date_part_up") { (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-29'); """ + sql """alter table lineitem_range_date modify column l_comment set stats ('row_count'='7');""" + def get_part = { def mv_name -> def part_res = sql """show partitions from ${mv_name}""" return part_res.size() diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy index 8bf629d93841fb..88d8ad6ea2d84d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_date_part_up_rewrite.groovy @@ -113,6 +113,9 @@ suite("mtmv_range_date_part_up_rewrite") { (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-31'); """ + sql """alter table lineitem_range_date_union modify column l_comment set stats ('row_count'='7');""" + sql """alter table orders_range_date_union modify column o_comment set stats ('row_count'='10');""" + sql """DROP MATERIALIZED VIEW if exists ${mv_prefix}_mv1;""" sql """CREATE MATERIALIZED VIEW ${mv_prefix}_mv1 BUILD IMMEDIATE REFRESH AUTO ON MANUAL partition by(date_trunc(`col1`, 'month')) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS select date_trunc(`l_shipdate`, 'day') as col1, l_shipdate, l_orderkey from lineitem_range_date_union as t1 left join orders_range_date_union as t2 on t1.l_orderkey = t2.o_orderkey group by col1, l_shipdate, l_orderkey;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy index b7e9562249d789..140a91edd7ca96 100644 --- a/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/create_part_and_up/range_datetime_part_up_rewrite.groovy @@ -98,6 +98,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-29 02:00:00'), (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-29 00:00:00'); """ + sql """alter table lineitem_range_datetime_union modify column l_comment set stats ('row_count'='7');""" sql """ insert into orders_range_datetime_union values @@ -112,6 +113,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-29 00:00:00'), (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-29 02:00:00'); """ + sql """alter table orders_range_datetime_union modify column o_comment set stats ('row_count'='10');""" sql """DROP MATERIALIZED VIEW if exists ${mv_prefix}_mv1;""" sql """CREATE MATERIALIZED VIEW ${mv_prefix}_mv1 BUILD IMMEDIATE REFRESH AUTO ON MANUAL partition by(date_trunc(`col1`, 'month')) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') AS @@ -166,7 +168,7 @@ suite("mtmv_range_datetime_part_up_rewrite") { for (int i = 0; i < mv_name_list.size(); i++) { def job_name = getJobName(db, mv_name_list[i]) waitingMTMVTaskFinished(job_name) - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } @@ -174,39 +176,47 @@ suite("mtmv_range_datetime_part_up_rewrite") { sql """alter table lineitem_range_datetime_union add partition p4 values [("2023-11-29 03:00:00"), ("2023-11-29 04:00:00"));""" sql """insert into lineitem_range_datetime_union values (1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-11-29 03:00:00')""" + sql """alter table lineitem_range_datetime_union modify column l_comment set stats ('row_count'='8');""" for (int i = 0; i < mv_name_list.size(); i++) { - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + // both mv should rewrite success + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } for (int i = 0; i < mv_name_list.size(); i++) { sql """refresh MATERIALIZED VIEW ${mv_name_list[i]} auto;""" - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + // both mv should rewrite success + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } sql """insert into lineitem_range_datetime_union values (3, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-11-29 03:00:00');""" + sql """alter table lineitem_range_datetime_union modify column l_comment set stats ('row_count'='9');""" for (int i = 0; i < mv_name_list.size(); i++) { - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + // both mv should rewrite success + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } for (int i = 0; i < mv_name_list.size(); i++) { sql """refresh MATERIALIZED VIEW ${mv_name_list[i]} auto;""" - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + // both mv should rewrite success + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } sql """ALTER TABLE lineitem_range_datetime_union DROP PARTITION IF EXISTS p4 FORCE""" for (int i = 0; i < mv_name_list.size(); i++) { - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + // both mv should rewrite success + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } for (int i = 0; i < mv_name_list.size(); i++) { sql """refresh MATERIALIZED VIEW ${mv_name_list[i]} auto;""" - mv_rewrite_success(query_stmt_list[i], mv_name_list[i]) + // both mv should rewrite success + mv_rewrite_any_success(query_stmt_list[i], mv_name_list) compare_res(query_stmt_list[i] + " order by 1,2,3") } diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy index f2b5ad09c0b17a..3aed3b0f9e24df 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy @@ -98,6 +98,9 @@ suite("partition_mv_rewrite_dimension_1") { (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); """ + sql """alter table orders_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_1 modify column l_comment set stats ('row_count'='7');""" + sql """analyze table orders_1 with sync;""" sql """analyze table lineitem_1 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy index 53f8777a1589d7..c7ee359cdef2e4 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_3.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_3") { sql """analyze table orders_2_3 with sync;""" sql """analyze table lineitem_2_3 with sync;""" + sql """alter table orders_2_3 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_3 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy index deb6255119a26e..e59b2771dd4e57 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_4.groovy @@ -124,6 +124,10 @@ suite("partition_mv_rewrite_dimension_2_4") { (3, null, 1, 99.5, 'yy'); """ + sql """alter table orders_2_4 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_4 modify column l_comment set stats ('row_count'='7');""" + sql """alter table partsupp_2_4 modify column ps_comment set stats ('row_count'='3');""" + sql """analyze table orders_2_4 with sync;""" sql """analyze table lineitem_2_4 with sync;""" sql """analyze table partsupp_2_4 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy index 5d8a6b5270285b..e5e43073bede3e 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_5.groovy @@ -124,6 +124,10 @@ suite("partition_mv_rewrite_dimension_2_5") { (3, null, 1, 99.5, 'yy'); """ + sql """alter table orders_2_5 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_5 modify column l_comment set stats ('row_count'='7');""" + sql """alter table partsupp_2_5 modify column ps_comment set stats ('row_count'='3');""" + sql """analyze table orders_2_5 with sync;""" sql """analyze table lineitem_2_5 with sync;""" sql """analyze table partsupp_2_5 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy index 1831e8de9a7a9d..f3289b007a0505 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_6.groovy @@ -124,6 +124,12 @@ suite("partition_mv_rewrite_dimension_2_6") { (3, null, 1, 99.5, 'yy'); """ + sql """alter table orders_2_6 modify column o_comment set stats ('row_count'='10');""" + + sql """alter table lineitem_2_6 modify column l_comment set stats ('row_count'='7');""" + + sql """alter table partsupp_2_6 modify column ps_comment set stats ('row_count'='3');""" + sql """analyze table orders_2_6 with sync;""" sql """analyze table lineitem_2_6 with sync;""" sql """analyze table partsupp_2_6 with sync;""" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy index 6def436e11c5c4..e36a5e277daca7 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy @@ -102,6 +102,8 @@ suite("partition_mv_rewrite_dimension_2_full_join") { sql """analyze table orders_2_full_join with sync;""" sql """analyze table lineitem_2_full_join with sync;""" + sql """alter table orders_2_full_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_full_join modify column l_comment set stats ('row_count'='7');""" def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy index 526e05f0692fbe..a615c7316bdb56 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_2") { sql """analyze table orders_2_2 with sync;""" sql """analyze table lineitem_2_2 with sync;""" + sql """alter table orders_2_2 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_2 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy index 0db7b3bf008c68..35d48270574a6f 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_anti_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_left_anti_join") { sql """analyze table orders_2_left_anti_join with sync;""" sql """analyze table lineitem_2_left_anti_join with sync;""" + sql """alter table orders_2_left_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_left_anti_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy index f59dffd0091bb4..616707412954e4 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_1") { sql """analyze table orders_2_1 with sync;""" sql """analyze table lineitem_2_1 with sync;""" + sql """alter table orders_2_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_1 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy index 5e60250ef91d26..f203db1134f2f2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_semi_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_left_semi_join") { sql """analyze table orders_2_left_semi_join with sync;""" sql """analyze table lineitem_2_left_semi_join with sync;""" + sql """alter table orders_2_left_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_left_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy index 289ed57287f1d8..c6097d845f41c9 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_anti_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_right_anti_join") { sql """analyze table orders_2_right_anti_join with sync;""" sql """analyze table lineitem_2_right_anti_join with sync;""" + sql """alter table orders_2_right_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_right_anti_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy index 8d6abb6dc561f8..850de42fc696eb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_right_join") { sql """analyze table orders_2_right_join with sync;""" sql """analyze table lineitem_2_right_join with sync;""" + sql """alter table orders_2_right_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_right_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy index 8f1f6f48a0ae91..97ddcd0eb1da29 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_semi_join.groovy @@ -102,6 +102,9 @@ suite("partition_mv_rewrite_dimension_2_right_semi_join") { sql """analyze table orders_2_right_semi_join with sync;""" sql """analyze table lineitem_2_right_semi_join with sync;""" + sql """alter table orders_2_right_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_2_right_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy index f433e18f48c062..776f1b996f1826 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_self_conn.groovy @@ -101,6 +101,9 @@ suite("partition_mv_rewrite_dimension_self_conn") { sql """analyze table orders_self_conn with sync;""" sql """analyze table lineitem_self_conn with sync;""" + sql """alter table orders_self_conn modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_self_conn modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy index dee43f20f4926a..fd41789d02661e 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_2_join_agg.groovy @@ -101,6 +101,9 @@ suite("dimension_2_join_agg_replenish") { sql """analyze table orders with sync;""" sql """analyze table lineitem with sync;""" + sql """alter table orders modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt @@ -649,19 +652,19 @@ suite("dimension_2_join_agg_replenish") { """ def right_semi_mv_stmt_8 = """select t1.l_orderkey, t1.l_shipdate, t1.l_partkey from orders - right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1 from lineitem where l_orderkey > 1 + 1 group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 + right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1 from lineitem where l_shipdate >= "2023-10-17" group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 on t1.l_orderkey = orders.o_orderkey group by t1.l_orderkey, t1.l_shipdate, t1.l_partkey """ def right_semi_mv_stmt_9 = """select t1.l_orderkey, t1.l_shipdate, t1.l_partkey from orders - right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1, count(*) as col2 from lineitem where l_orderkey > 1 + 1 group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 + right semi join (select l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate, sum(l_quantity) as col1, count(*) as col2 from lineitem where l_shipdate >= "2023-10-17" group by l_orderkey, l_partkey, l_suppkey, l_quantity, l_shipdate) as t1 on t1.l_orderkey = orders.o_orderkey group by t1.l_orderkey, t1.l_shipdate, t1.l_partkey """ - def right_semi_mv_stmt_10 = """select t1.l_orderkey + def right_semi_mv_stmt_10 = """select t1.sum_total, max_total+min_total as col3, count_all from orders right semi join (select l_orderkey, sum(l_quantity) as sum_total, max(l_quantity) as max_total, @@ -670,10 +673,10 @@ suite("dimension_2_join_agg_replenish") { sum(l_quantity) + count(*) as col5, bitmap_union(to_bitmap(case when l_quantity > 1 and l_orderkey IN (1, 3) then l_partkey else null end)) as cnt_1, bitmap_union(to_bitmap(case when l_quantity > 2 and l_orderkey IN (2) then l_partkey else null end)) as cnt_2 - from lineitem where l_orderkey > 1 + 1 group by l_orderkey) as t1 + from lineitem where l_shipdate >= "2023-10-17" group by l_orderkey) as t1 on t1.l_orderkey = orders.o_orderkey group by - t1.l_orderkey + t1.sum_total, col3, count_all """ def left_anti_mv_stmt_1 = """select t1.o_orderdate, t1.o_shippriority, t1.o_orderkey @@ -790,7 +793,7 @@ suite("dimension_2_join_agg_replenish") { bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 from orders where o_orderkey >= 1 + 1 group by o_orderkey) as t1 right anti join lineitem on lineitem.l_orderkey = t1.o_orderkey - group by l_orderkey, l_shipdate, l_partkey + group by l_orderkey, l_shipdate, l_partkey """ def right_anti_mv_stmt_6 = """select t1.l_shipdate, t1.l_quantity, t1.l_orderkey diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy index 04bd6ac5c6ac8d..3afbe42fd2a1c3 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_2_join_agg/dimension_join_agg_negative.groovy @@ -101,6 +101,9 @@ suite("dimension_join_agg_negative") { sql """analyze table orders_negative with sync;""" sql """analyze table lineitem_negative with sync;""" + sql """alter table orders_negative modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_negative modify column l_comment set stats ('row_count'='7');""" + // left join + agg (function + group by + +-*/ + filter) def left_mv_stmt_1 = """select t1.o_orderdate, t1.o_shippriority, t1.o_orderkey from (select o_orderkey, o_custkey, o_orderstatus, o_orderdate, o_shippriority from orders_negative group by o_orderkey, o_custkey, o_orderstatus, o_orderdate, o_shippriority) as t1 diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy index 763880ea94be44..2ca4a524989c99 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_equal/filter_equal_or_notequal.groovy @@ -99,6 +99,9 @@ suite("filter_equal_or_notequal_case") { sql """analyze table orders_1 with sync;""" sql """analyze table lineitem_1 with sync;""" + sql """alter table orders_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_1 modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy index ce30fab1b6c71e..751e5d0056726f 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/full_join_filter.groovy @@ -105,6 +105,9 @@ suite("full_join_filter") { sql """analyze table orders_full_join with sync;""" sql """analyze table lineitem_full_join with sync;""" + sql """alter table orders_full_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_full_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy index 932c2608ad408a..8e6f06036aa7da 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/inner_join_filter.groovy @@ -105,6 +105,9 @@ suite("inner_join_filter") { sql """analyze table orders_inner_join with sync;""" sql """analyze table lineitem_inner_join with sync;""" + sql """alter table orders_inner_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_inner_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy index d813e55f33286a..6522d7f2dc248a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_anti_join_filter.groovy @@ -107,6 +107,9 @@ suite("left_anti_join_filter") { sql """analyze table orders_left_anti_join with sync;""" sql """analyze table lineitem_left_anti_join with sync;""" + sql """alter table orders_left_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left_anti_join modify column l_comment set stats ('row_count'='9');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy index 608aec88c63621..37c9d3bfde4e47 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_join_filter.groovy @@ -105,6 +105,9 @@ suite("left_join_filter") { sql """analyze table orders_left_join with sync;""" sql """analyze table lineitem_left_join with sync;""" + sql """alter table orders_left_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy index da12c8fbeaef3f..2c9854eca82106 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/left_semi_join_filter.groovy @@ -105,6 +105,9 @@ suite("left_semi_join_filter") { sql """analyze table orders_left_semi_join with sync;""" sql """analyze table lineitem_left_semi_join with sync;""" + sql """alter table orders_left_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy index b920b766b07aa4..547a36c424c8cb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_anti_join_filter.groovy @@ -107,6 +107,9 @@ suite("right_anti_join_filter") { sql """analyze table orders_right_anti_join with sync;""" sql """analyze table lineitem_right_anti_join with sync;""" + sql """alter table orders_right_anti_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right_anti_join modify column l_comment set stats ('row_count'='9');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy index f3223855173f48..c29effd36cb701 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_join_filter.groovy @@ -105,6 +105,9 @@ suite("right_join_filter") { sql """analyze table orders_right_join with sync;""" sql """analyze table lineitem_right_join with sync;""" + sql """alter table orders_right_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy index bbf4e60afb64ec..6dc16554b26ca8 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension_predicate/right_semi_join_filter.groovy @@ -107,6 +107,9 @@ suite("right_semi_join_filter") { sql """analyze table orders_right_semi_join with sync;""" sql """analyze table lineitem_right_semi_join with sync;""" + sql """alter table orders_right_semi_join modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right_semi_join modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy b/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy index f4fc91db26c3b8..0aaaebb64cf3a3 100644 --- a/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/direct_query/direct_query.groovy @@ -131,6 +131,9 @@ suite("direct_query_mv") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='15');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" create_async_mv(db, "mv1_0", """ diff --git a/regression-test/suites/nereids_rules_p0/mv/dml/rewrite/rewrite_when_dml.groovy b/regression-test/suites/nereids_rules_p0/mv/dml/rewrite/rewrite_when_dml.groovy new file mode 100644 index 00000000000000..c1c00d448635ee --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/dml/rewrite/rewrite_when_dml.groovy @@ -0,0 +1,291 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("rewrite_when_dml") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set enable_agg_state=true" + + sql """ + drop table if exists orders + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table partsupp with sync""" + sql """analyze table lineitem with sync""" + sql """analyze table orders with sync""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + + // rewrite should success when table add column + def mv1_0 = """ + select + o_orderstatus, + l_partkey, + l_suppkey, + sum_union(sum_state(o_shippriority)), + group_concat_union(group_concat_state(o_orderstatus)), + avg_union(avg_state(l_linenumber)), + max_by_union(max_by_state(O_COMMENT,o_totalprice)), + count_union(count_state(l_orderkey)), + multi_distinct_count_union(multi_distinct_count_state(l_shipmode)) + from lineitem + left join orders + on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus, + l_partkey, + l_suppkey; + """ + def query1_0 = """ + select + o_orderstatus, + l_suppkey, + sum_merge(sum_state(o_shippriority)), + group_concat_merge(group_concat_state(o_orderstatus)), + avg_merge(avg_state(l_linenumber)), + max_by_merge(max_by_state(O_COMMENT,o_totalprice)), + count_merge(count_state(l_orderkey)), + multi_distinct_count_merge(multi_distinct_count_state(l_shipmode)) + from lineitem + left join orders + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus, + l_suppkey + order by o_orderstatus; + """ + order_qt_query1_0_before "${query1_0}" + sql """ALTER TABLE orders ADD COLUMN new_col INT KEY DEFAULT "0";""" + async_mv_rewrite_success(db, mv1_0, query1_0, "mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" + + + // rewrite should fail when table drop column which is used by query + def mv2_0 = """ + select + o_orderstatus, + l_partkey, + l_suppkey, + sum_union(sum_state(o_shippriority)), + group_concat_union(group_concat_state(o_orderstatus)), + avg_union(avg_state(l_linenumber)), + max_by_union(max_by_state(O_COMMENT,o_totalprice)), + count_union(count_state(l_orderkey)), + multi_distinct_count_union(multi_distinct_count_state(l_shipmode)) + from lineitem + left join orders + on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus, + l_partkey, + l_suppkey; + """ + def query2_0 = """ + select + o_orderstatus, + l_suppkey, + sum_merge(sum_state(o_shippriority)), + group_concat_merge(group_concat_state(o_orderstatus)), + avg_merge(avg_state(l_linenumber)), + max_by_merge(max_by_state(O_COMMENT,o_totalprice)), + count_merge(count_state(l_orderkey)), + multi_distinct_count_merge(multi_distinct_count_state(l_shipmode)) + from lineitem + left join orders + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus, + l_suppkey + order by o_orderstatus; + """ + order_qt_query2_0_before "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + sql """ + CREATE MATERIALIZED VIEW mv2_0 + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv2_0} + """ + waitingMTMVTaskFinishedByMvName("mv2_0") + sql """ALTER TABLE lineitem DROP COLUMN l_suppkey;""" + + try { + mv_not_part_in(query2_0, "mv2_0") + } catch (Exception e) { + // Unknown column because drop + } + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + // rewrite should fail when table drop column which is not used by query + // can be optimized in the future + def mv3_0 = """ + select + o_orderstatus, + l_partkey, + sum_union(sum_state(o_shippriority)), + group_concat_union(group_concat_state(o_orderstatus)), + avg_union(avg_state(l_linenumber)), + count_union(count_state(l_orderkey)), + multi_distinct_count_union(multi_distinct_count_state(l_shipmode)) + from lineitem + left join orders + on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus, + l_partkey; + """ + def query3_0 = """ + select + o_orderstatus, + sum_merge(sum_state(o_shippriority)), + group_concat_merge(group_concat_state(o_orderstatus)), + avg_merge(avg_state(l_linenumber)), + count_merge(count_state(l_orderkey)), + multi_distinct_count_merge(multi_distinct_count_state(l_shipmode)) + from lineitem + left join orders + on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + o_orderstatus + order by o_orderstatus; + """ + order_qt_query3_0_before "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" + sql """ + CREATE MATERIALIZED VIEW mv3_0 + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv3_0} + """ + waitingMTMVTaskFinishedByMvName("mv3_0") + sql """ALTER TABLE orders DROP COLUMN O_COMMENT;""" + mv_not_part_in(query3_0, "mv3_0") + + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" +} diff --git a/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy b/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy index ec8cecbaf517d0..4655f9d004bc7a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/grouping_sets/grouping_sets.groovy @@ -127,6 +127,9 @@ suite("materialized_view_grouping_sets") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table orders modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='7');""" + // query has group sets, and mv doesn't // single table grouping sets without grouping scalar function def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy b/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy new file mode 100644 index 00000000000000..15d93e32f65dc2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/is_in_debug_mode/is_in_debug_mode.groovy @@ -0,0 +1,156 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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.junit.Assert + +suite("is_in_debug_mode") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + UNIQUE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + create_async_mv(db, "basic_mv", """ + select * from orders where o_orderkey > 1; + """) + + sql """set skip_delete_sign = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_sign = false;""" + + + sql """set skip_storage_engine_merge = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_storage_engine_merge = false;""" + + + sql """set skip_delete_bitmap = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_bitmap = false;""" + + + sql """set skip_delete_predicate = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set skip_delete_predicate = false;""" + + + sql """set show_hidden_columns = true;""" + mv_not_part_in("""select * from orders where o_orderkey > 1;""", "basic_mv") + try { + sql """ + CREATE MATERIALIZED VIEW test_create_mv + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS select * from orders where o_orderkey > 2; + """ + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("because is in debug mode")) + } + sql """set show_hidden_columns = false;""" + + sql """drop materialized view if exists basic_mv""" +} + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy index 2b018ebc37373b..8c0674faeb8c32 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy @@ -123,6 +123,10 @@ suite("inner_join_dphyp") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy index 45d9aee067c798..71154fbeb276cf 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy @@ -123,6 +123,10 @@ suite("outer_join_dphyp") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy index b745f7d0197c28..3aebf92b7cec71 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy @@ -122,6 +122,10 @@ suite("inner_join") { sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // without filter def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy index faa2c747a837c5..cdc4632ff7379b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy @@ -278,9 +278,20 @@ suite("outer_join") { sql """analyze table lineitem with sync;""" sql """analyze table orders with sync;""" sql """analyze table partsupp with sync;""" + sql """analyze table lineitem_null with sync;""" + sql """analyze table orders_null with sync;""" sql """analyze table orders_same_col with sync;""" sql """analyze table lineitem_same_col with sync;""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table lineitem_same_col modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table orders_same_col modify column o_comment set stats ('row_count'='18');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + sql """alter table lineitem_null modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders_null modify column o_comment set stats ('row_count'='5');""" + + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy index 5688c2bbe21d6a..cce090b2a940fb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy @@ -106,6 +106,9 @@ suite("inner_join_infer_and_derive") { sql """analyze table orders_inner with sync;""" sql """analyze table lineitem_inner with sync;""" + sql """alter table orders_inner modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_inner modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy index dbfaa783383af8..7e51847c3b1a07 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/left_join_infer_and_derive.groovy @@ -106,6 +106,8 @@ suite("left_join_infer_and_derive") { sql """analyze table orders_left with sync;""" sql """analyze table lineitem_left with sync;""" + sql """alter table orders_left modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_left modify column l_comment set stats ('row_count'='7');""" def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy index 4b5bdf467a5d02..1761efa5ef4475 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/right_join_infer_and_derive.groovy @@ -106,6 +106,9 @@ suite("right_join_infer_and_derive") { sql """analyze table orders_right with sync;""" sql """analyze table lineitem_right with sync;""" + sql """alter table orders_right modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_right modify column l_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt diff --git a/regression-test/suites/nereids_rules_p0/mv/micro_test/micro_test_when_cte.groovy b/regression-test/suites/nereids_rules_p0/mv/micro_test/micro_test_when_cte.groovy new file mode 100644 index 00000000000000..9e80ea966fc1a2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/micro_test/micro_test_when_cte.groovy @@ -0,0 +1,204 @@ +package mv.micro_test +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("micro_test_when_cte") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders + """ + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate)( + FROM ('2023-12-01') TO ('2023-12-31') INTERVAL 1 DAY + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem + """ + sql""" + CREATE TABLE IF NOT EXISTS lineitem ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) + (FROM ('2023-12-01') TO ('2023-12-31') INTERVAL 1 DAY) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp + """ + sql """ + CREATE TABLE IF NOT EXISTS partsupp ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table partsupp with sync""" + sql """analyze table lineitem with sync""" + sql """analyze table orders with sync""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column O_COMMENT set stats ('row_count'='18');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + + def query_sql = """ + WITH scan_data_cte as ( + select t1.l_shipdate, t1.L_LINENUMBER, orders.O_CUSTKEY, l_suppkey + from (select * from lineitem where L_LINENUMBER > 1) t1 + left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY + ) + SELECT * FROM scan_data_cte; + """ + def mv_sql = """ + WITH scan_data_cte as ( + select t1.l_shipdate, t1.L_LINENUMBER, orders.O_CUSTKEY, l_suppkey + from (select * from lineitem where L_LINENUMBER > 1) t1 + left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY + ) + SELECT * FROM scan_data_cte; + """ + def mv_name = """mv_with_cte_test""" + + // query directly + order_qt_query_0_after "${query_sql}" + + // create and build complete mv + create_async_mv(db, mv_name, mv_sql) + // refresh mv complete + sql """refresh materialized view ${mv_name} complete""" + // query mv directly + waitingMTMVTaskFinishedByMvName(mv_name) + order_qt_query_mv_0 "select * from ${mv_name}" + + // create and build partition mv + create_async_partition_mv(db, mv_name, mv_sql, "(l_shipdate)") + + // refresh mv partly + sql """refresh materialized view ${mv_name} partitions(p_20231208_20231209)""" + // query mv directly + waitingMTMVTaskFinishedByMvName(mv_name) + order_qt_query_mv_1 "select * from ${mv_name}" + + // query rewrite + mv_rewrite_success(mv_sql, mv_name) + order_qt_query_0_after "${query_sql}" + + // DML + // base table insert into data when not partition table + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'); + """ + sql """refresh materialized view ${mv_name} complete""" + // query mv directly + waitingMTMVTaskFinishedByMvName(mv_name) + order_qt_query_mv_2 "select * from ${mv_name}" + + // base table insert into data when partition table + sql """ + insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'); + """ + sql """refresh materialized view ${mv_name} partitions(p_20231210_20231211)""" + // query mv directly + waitingMTMVTaskFinishedByMvName(mv_name) + order_qt_query_mv_3 "select * from ${mv_name}" +} diff --git a/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy b/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy index ba23c29ab9abc4..ea45b806529d8d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/negative/negative_test.groovy @@ -127,6 +127,9 @@ suite("negative_partition_mv_rewrite") { sql """analyze table lineitem_1 with sync;""" sql """analyze table partsupp_1 with sync;""" + sql """alter table orders_1 modify column o_comment set stats ('row_count'='10');""" + sql """alter table lineitem_1 modify column l_comment set stats ('row_count'='7');""" + def mv_name = "mv_1" def mtmv_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey diff --git a/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy b/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy index 3a130d489eb036..692ea03fc61806 100644 --- a/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy @@ -199,6 +199,10 @@ suite("nested_materialized_view") { (2, 3, 10, 11.01, 'supply2'); """ + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // simple nested materialized view def mv1_0_inner_mv = """ select diff --git a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy index 700c17606d2539..4870ec99e659c0 100644 --- a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy @@ -96,12 +96,20 @@ suite("nested_mtmv") { insert into orders_1 values (null, 1, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), (1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); """ @@ -109,22 +117,29 @@ suite("nested_mtmv") { sql """ insert into lineitem_1 values (null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (1, 1, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (3, 3, 3, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (3, 3, 3, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), (1, 2, 3, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (2, 1, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (2, 1, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), (3, 1, 3, 1, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), (1, 2, 1, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), (2, 2, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), (3, 3, 3, 3, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (3, 3, 3, 3, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), (1, 1, 1, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); """ sql""" insert into partsupp_1 values (1, 1, 1, 99.5, 'yy'), + (1, 1, 1, 99.5, 'yy'), + (2, 2, 2, 109.2, 'mm'), (2, 2, 2, 109.2, 'mm'), (3, 3, 1, 99.5, 'yy'), + (3, 3, 1, 99.5, 'yy'), (3, null, 1, 99.5, 'yy'); """ @@ -132,6 +147,10 @@ suite("nested_mtmv") { sql """analyze table lineitem_1 with sync;""" sql """analyze table partsupp_1 with sync;""" + sql """alter table orders_1 modify column o_orderdate set stats ('row_count'='17');""" + sql """alter table lineitem_1 modify column l_shipdate set stats ('row_count'='14');""" + sql """alter table partsupp_1 modify column ps_comment set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" def origin_res = sql stmt @@ -659,154 +678,77 @@ suite("nested_mtmv") { left join ( select t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 from ( - select t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, + t.agg1 as agg1, + t.sum_total as agg3, + t.max_total as agg4, + t.min_total as agg5, + t.count_all as agg6, + cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t1 - left join ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t2 - on t1.l_orderkey = t2.l_orderkey - where t1.l_orderkey > 1 - group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from lineitem_1 + inner join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where lineitem_1.l_shipdate >= "2023-10-17" + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey + ) as t + inner join partsupp_1 + on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey + where partsupp_1.ps_suppkey > 1 + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 ) as t1 left join ( - select t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, + t.agg1 as agg1, + t.sum_total as agg3, + t.max_total as agg4, + t.min_total as agg5, + t.count_all as agg6, + cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t1 - left join ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, - t.agg1 as agg1, - t.sum_total as agg3, - t.max_total as agg4, - t.min_total as agg5, - t.count_all as agg6, - cast(sum(IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0)) as decimal(28, 8)) as agg2 - from ( - select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, - sum(o_totalprice) as sum_total, - max(o_totalprice) as max_total, - min(o_totalprice) as min_total, - count(*) as count_all, - bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, - bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 - from lineitem_1 - inner join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where lineitem_1.l_shipdate >= "2023-10-17" - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey - ) as t - inner join partsupp_1 - on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey - where partsupp_1.ps_suppkey > 1 - group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 - ) as t2 - on t1.l_orderkey = t2.l_orderkey - where t1.l_orderkey > 1 - group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 + select l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, cast(sum(IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0)) as decimal(28, 8)) as agg1, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, + bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 + from lineitem_1 + inner join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where lineitem_1.l_shipdate >= "2023-10-17" + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey + ) as t + inner join partsupp_1 + on t.l_partkey = partsupp_1.ps_partkey and t.l_suppkey = partsupp_1.ps_suppkey + where partsupp_1.ps_suppkey > 1 + group by l_orderkey, l_partkey, l_suppkey, o_orderkey, o_custkey, ps_partkey, ps_suppkey, agg1, agg3, agg4, agg5, agg6 ) as t2 on t1.l_orderkey = t2.l_orderkey + where t1.l_orderkey > 1 group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 ) as t2 on t1.l_orderkey = t2.l_orderkey group by t1.l_orderkey, t2.l_partkey, t1.l_suppkey, t2.o_orderkey, t1.o_custkey, t2.ps_partkey, t1.ps_suppkey, t2.agg1, t1.agg2, t2.agg3, t1.agg4, t2.agg5, t1.agg6 """ - explain { - sql("${sql_2}") - contains "${mv_2}(${mv_2})" - } + mv_rewrite_any_success(sql_2, [mv_1, mv_2]) compare_res(sql_2 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") - explain { - sql("${sql_3}") - contains "${mv_3}(${mv_3})" - } + mv_rewrite_any_success(sql_3, [mv_3, mv_4]) compare_res(sql_3 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") - explain { - sql("${sql_4}") - contains "${mv_4}(${mv_4})" - } + mv_rewrite_any_success(sql_4, [mv_3, mv_4]) compare_res(sql_4 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") - explain { - sql("${sql_5}") - contains "${mv_5}(${mv_5})" - } + mv_rewrite_any_success(sql_5, [mv_3, mv_4, mv_5]) compare_res(sql_5 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13") } diff --git a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy index f9a84fa5250179..64354267a039bd 100644 --- a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv_switch/nested_mtmv_rewrite_switch.groovy @@ -99,6 +99,9 @@ suite("nested_mtmv_rewrite_switch") { sql """analyze table orders_2 with sync;""" sql """analyze table lineitem_2 with sync;""" + sql """alter table orders_2 modify column o_orderdate set stats ('row_count'='10');""" + sql """alter table lineitem_2 modify column l_shipdate set stats ('row_count'='7');""" + def compare_res = { def stmt -> sql "SET enable_materialized_view_rewrite=false" diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy index 68337b00d09da8..be5301cdf2ec17 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/anti/other_join_conjuncts_anti.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_anti") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" +sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // =, !=, >, <, <=, >= // left anti join other conjuncts in join condition def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy index b79e5808932044..0f07b7390cb3d7 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/inner/other_join_conjuncts_inner.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_inner") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // =, !=, >, <, <=, >= // other conjuncts in join condition def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy index 8a434364c89168..87fd620c704baf 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/outer/other_join_conjuncts_outer.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_outer") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // =, !=, >, <, <=, >= // left outer join // other conjuncts in join condition diff --git a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy index 30c21c77269e1c..4bbfa7e7c0a15c 100644 --- a/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/other_join_conjuncts/semi/other_join_conjuncts_semi.groovy @@ -140,6 +140,10 @@ suite("other_join_conjuncts_semi") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column lo_orderdate set stats ('row_count'='18');""" + sql """alter table lineitem modify column lo_orderdate set stats ('row_count'='10');""" +sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // left self join def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy index 35a983c264c0e2..f8023a9e1e4c49 100644 --- a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy @@ -480,6 +480,10 @@ suite("partition_mv_rewrite") { analyze table orders with sync; """ + sql """alter table orders modify column o_comment set stats ('row_count'='3');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='6');""" + sql """alter table lineitem_static modify column l_comment set stats ('row_count'='4');""" + // should rewrite successful when union rewrite enalbe if base table add new partition mv_rewrite_success(roll_up_all_partition_sql, "mv_10086", true, is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) diff --git a/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy b/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy index e518c016d2b641..4dda0f9c08414b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/same_name/sync_async_same_name.groovy @@ -57,6 +57,8 @@ suite("sync_async_same_name") { (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """analyze table orders with sync;""" def common_mv_name = 'common_mv_name' diff --git a/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy b/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy index 6b5c2dc45e875f..c42888e35abc0a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/scan/scan_table.groovy @@ -123,6 +123,10 @@ suite("mv_scan_table") { sql """analyze table lineitem with sync;""" sql """analyze table partsupp with sync;""" + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" +sql """alter table partsupp modify column ps_comment set stats ('row_count'='2');""" + // with filter def mv1_0 = """ diff --git a/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy b/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy index b6f44bba80139c..450fb9c0ea3187 100644 --- a/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/single_table_without_agg/single_table_without_aggregate.groovy @@ -52,6 +52,7 @@ suite("single_table_without_aggregate") { """ sql "analyze table orders with sync;" + sql """alter table orders modify column o_comment set stats ('row_count'='2');""" sql """set enable_stats=false;""" def check_rewrite = { mv_sql, query_sql, mv_name -> diff --git a/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy b/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy index e72e7575053b5b..192f9c420f18b5 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_all_compensate/union_all_compensate.groovy @@ -180,6 +180,9 @@ suite("union_all_compensate") { mv_rewrite_fail(query1_0, "test_agg_mv") order_qt_query1_1_after "${query1_0}" + sql """alter table test_table1 modify column num set stats ('row_count'='20');""" + sql """alter table test_table2 modify column num set stats ('row_count'='16');""" + // Aggregate, if query group by expression doesn't use the partition column, but the invalid partition is in the // grace_period, should not compensate union all, but should rewritten successfully diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy index 4f6717dfe59255..28a8f2b60f653c 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite.groovy @@ -78,6 +78,9 @@ suite ("partition_curd_union_rewrite") { ); """ + sql """alter table orders modify column o_comment set stats ('row_count'='3');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='3');""" + sql""" insert into orders values (1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'), diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy index 14170bf21f40fa..d82eaefde509e6 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/partition_curd_union_rewrite_hive.groovy @@ -155,6 +155,9 @@ suite ("partition_curd_union_rewrite_hive") { sql """create database if not exists ${db}""" sql """use ${db}""" + sql """alter table ${orders_tb_name} modify column o_comment set stats ('row_count'='3');""" +sql """alter table ${lineitem_tb_name} modify column l_comment set stats ('row_count'='3');""" + def mv_def_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy index 702431e5e391ae..4e74090b1a2bb2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite/usercase_union_rewrite.groovy @@ -90,6 +90,9 @@ suite ("usercase_union_rewrite") { sql """analyze table orders_user with sync;""" sql """analyze table lineitem_user with sync;""" + sql """alter table orders_user modify column o_comment set stats ('row_count'='4');""" + sql """alter table lineitem_user modify column l_comment set stats ('row_count'='3');""" + def create_mv_orders = { mv_name, mv_sql -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" sql """DROP TABLE IF EXISTS ${mv_name}""" diff --git a/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy b/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy index 5c64379950f942..cba2509483b6dc 100644 --- a/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/unsafe_equals/null_un_safe_equals.groovy @@ -57,6 +57,8 @@ suite("null_unsafe_equals") { (5, 2, 'o', 1.2, '2023-12-12', 'c','d', null, 'mi'); """ + sql """alter table orders modify column o_comment set stats ('row_count'='8');""" + def mv1_0 = """ select count(*), o_orderstatus, o_comment diff --git a/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy b/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy index 866f82af156c69..fae32e914ed031 100644 --- a/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy @@ -87,6 +87,9 @@ suite("variant_mv") { sql """analyze table github_events1 with sync;""" sql """analyze table github_events2 with sync;""" + sql """alter table github_events1 modify column created_at set stats ('row_count'='3');""" + sql """alter table github_events2 modify column created_at set stats ('row_count'='3');""" + // variant appear in where both slot and in expression def mv1_0 = """ SELECT diff --git a/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy b/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy index 400e113b6f3ae5..c51dea2f5a7403 100644 --- a/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/with_auth/with_select_table_auth.groovy @@ -112,6 +112,9 @@ suite("with_select_table_auth","p0,auth") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column o_comment set stats ('row_count'='18');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """grant select_priv on ${db}.orders to ${user_name}""" sql """grant select_priv on ${db}.lineitem to ${user_name}""" sql """grant select_priv on regression_test to ${user_name}""" diff --git a/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy b/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy index b7c6ecbd8ae008..757a4c42e5ad32 100644 --- a/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/with_sql_limit/query_with_sql_limit.groovy @@ -143,6 +143,10 @@ suite("query_with_sql_limit") { sql """analyze table lineitem with sync""" sql """analyze table orders with sync""" + sql """alter table orders modify column o_comment set stats ('row_count'='18');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='5');""" + sql """alter table partsupp modify column ps_comment set stats ('row_count'='3');""" + // test sql_select_limit default, default 9223372036854775807 sql """set sql_select_limit = 2;""" def mv1_0 = diff --git a/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy b/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy new file mode 100644 index 00000000000000..5ba2c05053e2a2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/with_table_operator/with_table_operator.groovy @@ -0,0 +1,195 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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("with_table_operator") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists orders; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + drop table if exists orders_partition; + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders_partition ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) ( + PARTITION `day_2` VALUES LESS THAN ('2023-12-9'), + PARTITION `day_3` VALUES LESS THAN ("2023-12-11"), + PARTITION `day_4` VALUES LESS THAN ("2023-12-30") + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into orders_partition values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + create_async_mv(db, "complete_agg_mv", """ + select count(*),o_orderdate + from orders + group by o_orderdate; + """) + + create_async_mv(db, "complete_mv", """ + select * from orders; + """) + + create_async_partition_mv(db, "partition_agg_mv", """ + select count(*),o_orderdate + from orders_partition + group by o_orderdate; + """, "(o_orderdate)") + + create_async_partition_mv(db, "partition_mv", """ + select * from orders_partition; + """, "(o_orderdate)") + + // query rewrite with tablet should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders TABLET(110) + group by o_orderdate; + """, ["complete_mv", "complete_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders TABLET(110);""", ["complete_mv", "complete_agg_mv"]) + + // query rewrite with index should fail + createMV(""" + create materialized view query_index_test + as + select o_orderdate, count(*) + from orders + group by o_orderdate; + """) + mv_rewrite_all_fail(""" + select * from orders index query_index_test; + """, ["complete_mv", "complete_agg_mv"]) + + // query rewrite with sample should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders TABLESAMPLE(20 percent) + group by o_orderdate; + """, ["complete_mv", "complete_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders TABLESAMPLE(20 percent);""", ["complete_mv", "complete_agg_mv"]) + + // query rewrite with partition should fail + mv_rewrite_all_fail(""" + select count(*),o_orderdate + from orders_partition PARTITION (day_2) + group by o_orderdate; + """, ["partition_mv", "partition_agg_mv"]) + + mv_rewrite_all_fail("""select * from orders_partition PARTITION (day_2);""", ["partition_mv", "partition_agg_mv"]) + + + sql """ + DROP MATERIALIZED VIEW IF EXISTS complete_agg_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS partition_agg_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS complete_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS partition_mv; + """ + sql """ + DROP MATERIALIZED VIEW IF EXISTS query_index_test on orders; + """ +} + diff --git a/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy b/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy index 98daa649dfb711..36636b1f535c1e 100644 --- a/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy +++ b/regression-test/suites/nereids_rules_p0/tablet_prune/test_tablet_prune.groovy @@ -21,7 +21,14 @@ suite("test_tablet_prune") { sql "drop table if exists t_customers_wide_index" sql """ - CREATE TABLE `t_customers_wide_index` ( `CUSTOMER_ID` int NULL, `ADDRESS` varchar(1500) NULL) ENGINE=OLAP UNIQUE KEY(`CUSTOMER_ID`) DISTRIBUTED BY HASH(`CUSTOMER_ID`) BUCKETS 32 PROPERTIES ( "replication_allocation" = "tag.location.default: 1", "file_cache_ttl_seconds" = "0", "is_being_synced" = "false", "storage_medium" = "hdd", "storage_format" = "V2", "inverted_index_storage_format" = "V2", "enable_unique_key_merge_on_write" = "true", "light_schema_change" = "true", "store_row_column" = "true", "row_store_page_size" = "16384", "disable_auto_compaction" = "false", "enable_single_replica_compaction" = "false", "group_commit_interval_ms" = "10000", "group_commit_data_bytes" = "134217728", "enable_mow_light_delete" = "false" ); """ + CREATE TABLE `t_customers_wide_index` + (`CUSTOMER_ID` int NULL,`ADDRESS` varchar(1500) NULL) + ENGINE=OLAP + UNIQUE KEY(`CUSTOMER_ID`) + DISTRIBUTED BY HASH(`CUSTOMER_ID`) + BUCKETS 32 + PROPERTIES ( "replication_allocation" = "tag.location.default: 1");""" + sql """ insert into t_customers_wide_index values (1, "111"); """ diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy index 76696e1c62b305..54ad59943d77bd 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/aggHaveDupBase.groovy @@ -36,6 +36,7 @@ suite ("aggHaveDupBase") { distributed BY hash(k1) buckets 3 properties("replication_num" = "1"); """ + sql """alter table agg_have_dup_base modify column k1 set stats ('row_count'='5');""" sql "insert into agg_have_dup_base select 1,1,1,'a';" sql "insert into agg_have_dup_base select 2,2,2,'b';" diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy index 91293bb747aa64..e2f334566e9c24 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/case_ignore.groovy @@ -36,6 +36,7 @@ suite ("case_ignore") { sql "insert into case_ignore select 2,2,2,'b';" sql "insert into case_ignore select 3,-3,null,'c';" + createMV ("create materialized view k12a as select K1,abs(K2) from case_ignore;") sleep(3000) @@ -55,6 +56,7 @@ suite ("case_ignore") { order_qt_select_mv "select K1,abs(K2) from case_ignore order by K1;" sql """set enable_stats=true;""" + sql """alter table case_ignore modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,abs(k2) from case_ignore order by k1;", "k12a") mv_rewrite_success("select K1,abs(K2) from case_ignore order by K1;", "k12a") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy index 132a4c3f054571..3537777d312c58 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_abs.groovy @@ -57,6 +57,7 @@ suite ("dup_gb_mv_abs") { order_qt_select_mv_sub "select sum(abs(k2)) from dup_gb_mv_abs group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_gb_mv_abs modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(abs(k2)) from dup_gb_mv_abs group by k1;", "k12sa") mv_rewrite_success("select sum(abs(k2)) from dup_gb_mv_abs group by k1;", "k12sa") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy index 45271415b98ef5..2e6e9c6b627850 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_gb_mv_plus.groovy @@ -57,6 +57,7 @@ suite ("dup_gb_mv_plus") { order_qt_select_mv_sub "select sum(k2+1) from dup_gb_mv_plus group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_gb_mv_plus modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,sum(k2+1) from dup_gb_mv_plus group by k1;", "k12sp") mv_rewrite_success("select sum(k2+1) from dup_gb_mv_plus group by k1;", "k12sp") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy index c4770d6dcd0262..6cef52fd96edc7 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_abs.groovy @@ -31,7 +31,6 @@ suite ("dup_mv_abs") { distributed BY hash(k1) buckets 3 properties("replication_num" = "1"); """ - sql "insert into dup_mv_abs select 1,1,1,'a';" sql "insert into dup_mv_abs select 2,2,2,'b';" sql "insert into dup_mv_abs select 3,-3,null,'c';" @@ -69,6 +68,7 @@ suite ("dup_mv_abs") { order_qt_select_group_mv_not "select sum(abs(k2)) from dup_mv_abs group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table dup_mv_abs modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,abs(k2) from dup_mv_abs order by k1;", "k12a") mv_rewrite_success("select abs(k2) from dup_mv_abs order by k1;", "k12a") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy index 788647ae224d98..7fd22f5b71892c 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bin.groovy @@ -69,6 +69,7 @@ suite ("dup_mv_bin") { order_qt_select_group_mv_not "select group_concat(bin(k2)) from dup_mv_bin group by k3 order by k3;" sql """set enable_stats=true;""" + sql """alter table dup_mv_bin modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,bin(k2) from dup_mv_bin order by k1;", "k12b") mv_rewrite_success("select bin(k2) from dup_mv_bin order by k1;", "k12b") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy index d049a0aeb7c1b0..f5784e7a5fdaa6 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_bm_hash.groovy @@ -45,6 +45,7 @@ suite ("dup_mv_bm_hash") { order_qt_select_mv "select bitmap_union_count(to_bitmap(k2)) from dup_mv_bm_hash group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_mv_bm_hash modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select bitmap_union_count(to_bitmap(k2)) from dup_mv_bm_hash group by k1 order by k1;", "dup_mv_bm_hash_mv1") createMV("create materialized view dup_mv_bm_hash_mv2 as select k1,bitmap_union(bitmap_hash(k3)) from dup_mv_bm_hash group by k1;") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy index b661e75ad967ab..d96d2e07843dd9 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_plus.groovy @@ -46,7 +46,6 @@ suite ("dup_mv_plus") { sql "analyze table dup_mv_plus with sync;" sql """set enable_stats=false;""" - order_qt_select_star "select * from dup_mv_plus order by k1;" mv_rewrite_success("select k1,k2+1 from dup_mv_plus order by k1;", "k12p") @@ -86,6 +85,7 @@ suite ("dup_mv_plus") { order_qt_select_mv "select k1,k2+1 from dup_mv_plus order by k2;" sql """set enable_stats=true;""" + sql """alter table dup_mv_plus modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,k2+1 from dup_mv_plus order by k1;", "k12p") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy index c5098c55074c7d..95902c27866481 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/dup_mv_year.groovy @@ -45,6 +45,7 @@ suite ("dup_mv_year") { order_qt_select_mv "select k1,year(k2) from dup_mv_year order by k1;" sql """set enable_stats=true;""" + sql """alter table dup_mv_year modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1,year(k2) from dup_mv_year order by k1;", "k12y") createMV "create materialized view k13y as select k1,year(k3) from dup_mv_year;" diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy index 7e317d501bea07..d0fa214e98f042 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot1.groovy @@ -52,5 +52,6 @@ suite ("multi_slot1") { order_qt_select_mv "select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot1 order by abs(k1)+k2+1,abs(k2+2)+k3+3;" sql """set enable_stats=true;""" + sql """alter table multi_slot1 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot1 order by abs(k1)+k2+1,abs(k2+2)+k3+3", "k1a2p2ap3p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy index 6bbded534a77ee..0fa36b77b44114 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot2.groovy @@ -65,6 +65,7 @@ suite ("multi_slot2") { order_qt_select_base "select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from multi_slot2 group by abs(k1)+k2 order by abs(k1)+k2;" sql """set enable_stats=true;""" + sql """alter table multi_slot2 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,sum(abs(k2+2)+k3+3) from multi_slot2 group by abs(k1)+k2+1 order by abs(k1)+k2+1", "k1a2p2ap3ps") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy index c6016296e8b352..51cc90aaf6ae26 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot3.groovy @@ -53,5 +53,6 @@ suite ("multi_slot3") { order_qt_select_mv "select k1+1,abs(k2+2)+k3+3 from multi_slot3 order by k1+1;" sql """set enable_stats=true;""" + sql """alter table multi_slot3 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select k1+1,abs(k2+2)+k3+3 from multi_slot3 order by k1+1;", "k1p2ap3p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy index a09f3937aee302..49aba77aad05ff 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy @@ -56,5 +56,6 @@ suite ("multi_slot4") { order_qt_select_mv "select k1+1,sum(abs(k2+2)+k3+3) from multi_slot4 group by k1+1 order by k1+1;" sql """set enable_stats=true;""" + sql """alter table multi_slot4 modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1+1,sum(abs(k2+2)+k3+3) from multi_slot4 group by k1+1 order by k1+1;", "k1p2ap3ps") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy index 55ea991913cb3e..ebe6c8e57800a4 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot5.groovy @@ -58,5 +58,6 @@ suite ("multi_slot5") { order_qt_select_mv "select k1,version() from multi_slot5 order by k1;" sql """set enable_stats=true;""" + sql """alter table multi_slot5 modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k2+k3 from multi_slot5 order by k1;", "k123p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy index 9201a1cea4d321..2ed66fc8477a37 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot6.groovy @@ -84,5 +84,6 @@ suite ("multi_slot6") { order_qt_select_mv "select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot6 order by abs(k1)+k2+1,abs(k2+2)+k3+3;" sql """set enable_stats=true;""" + sql """alter table multi_slot6 modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+k2+1,abs(k2+2)+k3+3 from multi_slot6 order by abs(k1)+k2+1,abs(k2+2)+k3+3", "k1a2p2ap3p") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy index 236bde3bf6c828..60e7f6fefed232 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/mv_with_view.groovy @@ -67,6 +67,7 @@ suite ("mv_with_view") { qt_select_mv "select * from v_k124 order by k1;" sql """set enable_stats=true;""" + sql """alter table d_table modify column k1 set stats ('row_count'='4');""" mv_rewrite_fail("select * from d_table order by k1;", "k312") sql """ diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy index d6cd0d5a24421a..387c155a5e45c8 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/single_slot.groovy @@ -47,7 +47,6 @@ suite ("single_slot") { sql "analyze table single_slot with sync;" sql """set enable_stats=false;""" - order_qt_select_star "select * from single_slot order by k1;" explain { @@ -58,5 +57,6 @@ suite ("single_slot") { order_qt_select_mv "select abs(k1)+1 t,sum(abs(k2+1)) from single_slot group by t order by t;" sql """set enable_stats=true;""" + sql """alter table single_slot modify column k1 set stats ('row_count'='4');""" mv_rewrite_success("select abs(k1)+1 t,sum(abs(k2+1)) from single_slot group by t order by t;", "k1ap2spa") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy index 031560a41ef502..be8bc50532ff81 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/sum_devide_count.groovy @@ -62,7 +62,7 @@ suite ("sum_devide_count") { order_qt_select_mv "select sum(k2)/count(k2) from sum_devide_count;" sql """set enable_stats=true;""" - + sql """alter table sum_devide_count modify column k1 set stats ('row_count'='5');""" mv_rewrite_success("select k1,k4,sum(k2)/count(k2) from sum_devide_count group by k1,k4 order by k1,k4;", "kavg") mv_rewrite_success("select k1,sum(k2)/count(k2) from sum_devide_count group by k1 order by k1;", "kavg") diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy index df390b26551054..1ebaa8884ab383 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/unique_mv.groovy @@ -47,6 +47,7 @@ suite ("unique_mv") { mv_rewrite_success("SELECT * FROM c5816_t WHERE call_uuid='adc';", "mv_1") sql """set enable_stats=true;""" + sql """alter table c5816_t modify column org_id set stats ('row_count'='1');""" mv_rewrite_success("SELECT * FROM c5816_t WHERE call_uuid='adc';", "mv_1") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy index 9a5e42034be3cb..2b6b0dd354e4cd 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/MVMultiUsage.groovy @@ -37,6 +37,7 @@ suite ("MVMultiUsage") { sql """insert into MVMultiUsage values("2020-01-02",2,"b",2,2,2);""" sql """insert into MVMultiUsage values("2020-01-03",3,"c",3,3,3);""" + createMV("create materialized view MVMultiUsage_mv as select deptno, empid, salary from MVMultiUsage order by deptno;") sleep(3000) @@ -58,6 +59,8 @@ suite ("MVMultiUsage") { order_qt_select_mv "select * from (select deptno, empid from MVMultiUsage where deptno>100) A join (select deptno, empid from MVMultiUsage where deptno >200) B using (deptno) order by 1;" sql """set enable_stats=true;""" + sql """alter table MVMultiUsage modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from MVMultiUsage order by empid;", "MVMultiUsage_mv") explain { sql("select * from (select deptno, empid from MVMultiUsage where deptno>100) A join (select deptno, empid from MVMultiUsage where deptno >200) B using (deptno);") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy index 246c97022e9e85..86a85dcc647443 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/MVWithAs.groovy @@ -30,6 +30,7 @@ suite ("MVWithAs") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into MVWithAs values("2020-01-01",1,"a",1);""" sql """insert into MVWithAs values("2020-01-01",1,"a",1);""" sql """insert into MVWithAs values("2020-01-01",1,"a",1);""" @@ -53,6 +54,8 @@ suite ("MVWithAs") { order_qt_select_mv "select count(tag_id) from MVWithAs t;" sql """set enable_stats=true;""" + sql """alter table MVWithAs modify column time_col set stats ('row_count'='7');""" + mv_rewrite_fail("select * from MVWithAs order by time_col;", "MVWithAs_mv") mv_rewrite_success("select count(tag_id) from MVWithAs t;", "MVWithAs_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy index 79ede78f0d3752..f45d523d423527 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggCDInBitmap.groovy @@ -34,7 +34,6 @@ suite ("aggCDInBitmap") { sql "analyze table aggCDInBitmap with sync;" sql """set enable_stats=false;""" - order_qt_select_star "select * from aggCDInBitmap order by 1;" @@ -45,9 +44,10 @@ suite ("aggCDInBitmap") { order_qt_select_mv "select k1, count(distinct v1) from aggCDInBitmap group by k1 order by k1;" sql """set enable_stats=true;""" + sql """alter table aggCDInBitmap modify column k1 set stats ('row_count'='3');""" explain { sql("select k1, count(distinct v1) from aggCDInBitmap group by k1;") contains "bitmap_union_count" } - + } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy index 1a7ccd851503d8..a234661150e732 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggMVCalcAggFun.groovy @@ -38,7 +38,6 @@ suite ("aggMVCalcAggFun") { sql """insert into aggMVCalcAggFun values("2020-01-02",2,"b",2,2,2);""" sql """insert into aggMVCalcAggFun values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view aggMVCalcAggFunMv as select deptno, empid, sum(salary) from aggMVCalcAggFun group by empid, deptno;") sleep(3000) @@ -55,6 +54,7 @@ suite ("aggMVCalcAggFun") { order_qt_select_mv "select deptno, sum(salary + 1) from aggMVCalcAggFun where deptno > 10 group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table aggMVCalcAggFun modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggMVCalcAggFun order by empid;", "aggMVCalcAggFunMv")\ mv_rewrite_fail("select deptno, sum(salary + 1) from aggMVCalcAggFun where deptno > 10 group by deptno;", "aggMVCalcAggFunMv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy index 0cb12cbcde0b51..c29eec40c8da93 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV1.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV1") { order_qt_select_mv "select sum(salary), deptno from aggOnAggMV1 group by deptno order by deptno;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV1 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV1 order by empid;", "aggOnAggMV1_mv") mv_rewrite_success("select sum(salary), deptno from aggOnAggMV1 group by deptno order by deptno;", "aggOnAggMV1_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy index bd826537ff88f7..e38a27c1db3c53 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV10.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV10") { order_qt_select_mv "select deptno, commission, sum(salary) + 1 from aggOnAggMV10 group by rollup (deptno, commission) order by 1,2;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV10 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV10 order by empid;", "aggOnAggMV10_mv") mv_rewrite_success("select deptno, commission, sum(salary) + 1 from aggOnAggMV10 group by rollup (deptno, commission);", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy index 793389ec68e4ae..b0f379b19476b8 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV11.groovy @@ -33,6 +33,7 @@ suite ("aggOnAggMV11") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into aggOnAggMV11 values("2020-01-01",1,"a",1,1,1);""" sql """insert into aggOnAggMV11 values("2020-01-02",2,"b",2,2,2);""" sql """insert into aggOnAggMV11 values("2020-01-03",3,"c",3,3,3);""" @@ -54,6 +55,8 @@ suite ("aggOnAggMV11") { order_qt_select_mv "select deptno, count(salary) + count(1) from aggOnAggMV11 group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV11 modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from aggOnAggMV11 order by empid;", "aggOnAggMV11_mv") mv_rewrite_fail("select deptno, count(salary) + count(1) from aggOnAggMV11 group by deptno;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy index 32d897939ea652..179e22584d32a1 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV2.groovy @@ -56,6 +56,7 @@ suite ("aggOnAggMV2") { order_qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV2 group by deptno) a where (sum_salary * 2) > 3 order by deptno ;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV2 modify column time_col set stats ('row_count'='3');""" mv_rewrite_fail("select * from aggOnAggMV2 order by empid;", "aggOnAggMV2_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV2 group by deptno) a where (sum_salary * 2) > 3 order by deptno ;", "aggOnAggMV2_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy index dfd0ce3e67503c..0b54345099bcb9 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV3.groovy @@ -56,6 +56,7 @@ suite ("aggOnAggMV3") { order_qt_select_mv "select commission, sum(salary) from aggOnAggMV3 where commission * (deptno + commission) = 100 group by commission order by commission;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV3 modify column time_col set stats ('row_count'='5');""" mv_rewrite_fail("select * from aggOnAggMV3 order by empid;", "aggOnAggMV3_mv") mv_rewrite_success("select commission, sum(salary) from aggOnAggMV3 where commission * (deptno + commission) = 100 group by commission order by commission;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy index 481d8f19b5a133..d90057b8bf5120 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV5.groovy @@ -33,6 +33,8 @@ suite ("aggOnAggMV5") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table aggOnAggMV5 modify column time_col set stats ('row_count'='4');""" + sql """insert into aggOnAggMV5 values("2020-01-01",1,"a",1,1,1);""" sql """insert into aggOnAggMV5 values("2020-01-02",2,"b",2,2,2);""" sql """insert into aggOnAggMV5 values("2020-01-03",3,"c",3,3,3);""" diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy index abd66e1ab13ccd..7e2e87718b8e76 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV6.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV6") { order_qt_select_mv "select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV6 where deptno>=20 group by deptno) a where sum_salary>10 order by 1;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV6 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV6 order by empid;", "aggOnAggMV6_mv") mv_rewrite_success("select * from (select deptno, sum(salary) as sum_salary from aggOnAggMV6 where deptno>=20 group by deptno) a where sum_salary>10;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy index 53b75d03d85167..386871a8a5a98f 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/aggOnAggMV7.groovy @@ -54,6 +54,7 @@ suite ("aggOnAggMV7") { order_qt_select_mv "select deptno, sum(salary) from aggOnAggMV7 where deptno>=20 group by deptno order by 1;" sql """set enable_stats=true;""" + sql """alter table aggOnAggMV7 modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from aggOnAggMV7 order by empid;", "aggOnAggMV7_mv") mv_rewrite_success("select deptno, sum(salary) from aggOnAggMV7 where deptno>=20 group by deptno;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy index 13e2097cdb46ba..8b2697cb1f78f7 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/bitmapUnionIn.groovy @@ -30,6 +30,7 @@ suite ("bitmapUnionIn") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into bitmapUnionIn values("2020-01-01",1,"a",1);""" sql """insert into bitmapUnionIn values("2020-01-02",2,"b",2);""" @@ -50,6 +51,8 @@ suite ("bitmapUnionIn") { order_qt_select_mv "select user_id, bitmap_union_count(to_bitmap(tag_id)) a from bitmapUnionIn group by user_id having a>1 order by a;" sql """set enable_stats=true;""" + sql """alter table bitmapUnionIn modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from bitmapUnionIn order by time_col;", "bitmapUnionIn_mv") mv_rewrite_success("select user_id, bitmap_union_count(to_bitmap(tag_id)) a from bitmapUnionIn group by user_id having a>1 order by a;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy index bfd7c159dade89..eb314fecccbb4b 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/distinctQuery.groovy @@ -37,6 +37,8 @@ suite ("distinctQuery") { sql """insert into distinctQuery values("2020-01-02",2,"b",2,2,2);""" sql """insert into distinctQuery values("2020-01-03",3,"c",3,3,3);""" + sql """alter table distinctQuery modify column time_col set stats ('row_count'='5');""" + createMV("create materialized view distinctQuery_mv as select deptno, count(salary) from distinctQuery group by deptno;") createMV("create materialized view distinctQuery_mv2 as select empid, deptno, count(salary) from distinctQuery group by empid, deptno;") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy index c28dac1d227aab..52abff46678bb0 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/incMVReInSub.groovy @@ -31,6 +31,7 @@ suite ("incMVReInSub") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into incMVReInSub values("2020-01-01",1,"a",1);""" sql """insert into incMVReInSub values("2020-01-02",2,"b",2);""" @@ -52,6 +53,8 @@ suite ("incMVReInSub") { order_qt_select_mv "select user_id, bitmap_union(to_bitmap(tag_id)) from incMVReInSub where user_name in (select user_name from incMVReInSub group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;" sql """set enable_stats=true;""" + sql """alter table incMVReInSub modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from incMVReInSub order by time_col;", "incMVReInSub_mv") mv_rewrite_fail("select user_id, bitmap_union(to_bitmap(tag_id)) from incMVReInSub where user_name in (select user_name from incMVReInSub group by user_name having bitmap_union_count(to_bitmap(tag_id)) >1 ) group by user_id order by user_id;", diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy index ee8e688c0e7313..6172773ca7a43b 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/incRewriteCD.groovy @@ -31,6 +31,7 @@ suite ("incRewriteCD") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into incRewriteCD values("2020-01-01",1,"a",1);""" sql """insert into incRewriteCD values("2020-01-02",2,"b",2);""" @@ -50,6 +51,8 @@ suite ("incRewriteCD") { order_qt_select_mv "select user_name, count(distinct tag_id) from incRewriteCD group by user_name order by user_name;" sql """set enable_stats=true;""" + sql """alter table incRewriteCD modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from incRewriteCD order by time_col;", "incRewriteCD_mv") mv_rewrite_fail("select user_name, count(distinct tag_id) from incRewriteCD group by user_name;", "incRewriteCD_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy index 8ed3e09bdf0df3..378731dbb97a62 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnCalcToJoin.groovy @@ -35,6 +35,8 @@ suite ("joinOnCalcToJoin") { sql """insert into joinOnCalcToJoin values("2020-01-03",3,"c",3,3,3);""" sql """insert into joinOnCalcToJoin values("2020-01-02",2,"b",2,7,2);""" + sql """alter table joinOnCalcToJoin modify column time_col set stats ('row_count'='3');""" + sql """ DROP TABLE IF EXISTS joinOnCalcToJoin_1; """ sql """ create table joinOnCalcToJoin_1 ( @@ -49,6 +51,7 @@ suite ("joinOnCalcToJoin") { sql """insert into joinOnCalcToJoin_1 values("2020-01-03",3,"c",3);""" sql """insert into joinOnCalcToJoin_1 values("2020-01-02",2,"b",1);""" + createMV("create materialized view joinOnLeftPToJoin_mv as select empid, deptno from joinOnCalcToJoin;") sleep(3000) createMV("create materialized view joinOnLeftPToJoin_1_mv as select deptno, cost from joinOnCalcToJoin_1;") @@ -62,6 +65,8 @@ suite ("joinOnCalcToJoin") { ["joinOnLeftPToJoin_mv", "joinOnLeftPToJoin_1_mv"]) sql """set enable_stats=true;""" + sql """alter table joinOnCalcToJoin_1 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_all_success("select * from (select empid, deptno from joinOnCalcToJoin where empid = 0) A join (select deptno, cost from joinOnCalcToJoin_1 where deptno > 0) B on A.deptno = B.deptno;", ["joinOnLeftPToJoin_mv", "joinOnLeftPToJoin_1_mv"]) } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy index f2f7a014d98fc8..69d1c39931c17f 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/joinOnLeftPToJoin.groovy @@ -31,6 +31,8 @@ suite ("joinOnLeftPToJoin") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """alter table joinOnLeftPToJoin modify column time_col set stats ('row_count'='3');""" + sql """insert into joinOnLeftPToJoin values("2020-01-02",2,"b",2,2,2);""" sql """insert into joinOnLeftPToJoin values("2020-01-03",3,"c",3,3,3);""" sql """insert into joinOnLeftPToJoin values("2020-01-02",2,"b",2,7,2);""" @@ -45,6 +47,7 @@ suite ("joinOnLeftPToJoin") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into joinOnLeftPToJoin_1 values("2020-01-02",2,"b",2);""" sql """insert into joinOnLeftPToJoin_1 values("2020-01-03",3,"c",3);""" sql """insert into joinOnLeftPToJoin_1 values("2020-01-02",2,"b",1);""" @@ -64,6 +67,8 @@ suite ("joinOnLeftPToJoin") { order_qt_select_mv "select * from (select deptno , sum(salary) from joinOnLeftPToJoin group by deptno) A join (select deptno, max(cost) from joinOnLeftPToJoin_1 group by deptno ) B on A.deptno = B.deptno order by A.deptno;" sql """set enable_stats=true;""" + sql """alter table joinOnLeftPToJoin_1 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_all_success("select * from (select deptno , sum(salary) from joinOnLeftPToJoin group by deptno) A join (select deptno, max(cost) from joinOnLeftPToJoin_1 group by deptno ) B on A.deptno = B.deptno;", ["joinOnLeftPToJoin_mv", "joinOnLeftPToJoin_1_mv"]) } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy index 698f4ae13f328d..e36ae20e9bcb11 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/onlyGroupBy.groovy @@ -33,6 +33,7 @@ suite ("onlyGroupBy") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into onlyGroupBy values("2020-01-01",1,"a",1,1,1);""" sql """insert into onlyGroupBy values("2020-01-02",2,"b",2,2,2);""" sql """insert into onlyGroupBy values("2020-01-03",3,"c",3,3,3);""" @@ -47,5 +48,7 @@ suite ("onlyGroupBy") { mv_rewrite_success("select deptno from onlyGroupBy group by deptno;", "onlyGroupBy_mv") sql """set enable_stats=true;""" + sql """alter table onlyGroupBy modify column time_col set stats ('row_count'='4');""" + mv_rewrite_success("select deptno from onlyGroupBy group by deptno;", "onlyGroupBy_mv") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy index 7f455da429e6c3..21864f973ee8c0 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/orderByOnPView.groovy @@ -54,6 +54,8 @@ suite ("orderByOnPView") { order_qt_select_mv "select empid from orderByOnPView order by deptno;" sql """set enable_stats=true;""" + sql """alter table orderByOnPView modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from orderByOnPView where time_col='2020-01-01' order by empid;", "orderByOnPView_mv") mv_rewrite_success("select empid from orderByOnPView where deptno = 0 order by deptno;", "orderByOnPView_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy index 384137471d827f..157a15498e717c 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV1.groovy @@ -33,6 +33,7 @@ suite ("projectMV1") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV1 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV1 values("2020-01-02",2,"b",2,2,2);""" @@ -52,6 +53,8 @@ suite ("projectMV1") { order_qt_select_mv "select empid, deptno from projectMV1 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV1 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV1 where time_col='2020-01-01' order by empid;", "projectMV1_mv") mv_rewrite_success("select empid, deptno from projectMV1 where deptno=0 order by empid;", "projectMV1_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy index 672bda198b01aa..4d85fcc9c8aeae 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV2.groovy @@ -33,6 +33,7 @@ suite ("projectMV2") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV2 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV2 values("2020-01-02",2,"b",2,2,2);""" @@ -55,6 +56,8 @@ suite ("projectMV2") { order_qt_select_base "select name from projectMV2 where deptno -1 = 0 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV2 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV2 order by empid;", "projectMV2_mv") mv_rewrite_success("select empid + 1 from projectMV2 where deptno = 1 order by empid;", "projectMV2_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy index 37b73a9a81aff4..9ad301fbbc6a97 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV3.groovy @@ -33,6 +33,7 @@ suite ("projectMV3") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV3 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV3 values("2020-01-02",2,"b",2,2,2);""" @@ -57,6 +58,8 @@ suite ("projectMV3") { order_qt_select_mv2 "select name from projectMV3 where deptno -1 = 0 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV3 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV3 order by empid;", "projectMV3_mv") mv_rewrite_success("select empid + 1, name from projectMV3 where deptno = 1 order by empid;", "projectMV3_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy index ea87551dfa1249..ddcdada549ad8f 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/projectMV4.groovy @@ -33,6 +33,7 @@ suite ("projectMV4") { partition by range (time_col) (partition p1 values less than MAXVALUE) distributed by hash(time_col) buckets 3 properties('replication_num' = '1'); """ + sql """insert into projectMV4 values("2020-01-01",1,"a",1,1,1);""" sql """insert into projectMV4 values("2020-01-02",2,"b",2,2,2);""" @@ -57,6 +58,8 @@ suite ("projectMV4") { order_qt_select_base "select empid from projectMV4 where deptno > 1 and empid > 1 order by empid;" sql """set enable_stats=true;""" + sql """alter table projectMV4 modify column time_col set stats ('row_count'='3');""" + mv_rewrite_fail("select * from projectMV4 order by empid;", "projectMV4_mv") mv_rewrite_success("select name from projectMV4 where deptno > 1 and salary > 1 and name = 'a' order by name;", "projectMV4_mv") diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy index 25300c9d602681..67312eed088ed4 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/subQuery.groovy @@ -37,7 +37,6 @@ suite ("subQuery") { sql """insert into subQuery values("2020-01-02",2,"b",2,2,2);""" sql """insert into subQuery values("2020-01-03",3,"c",3,3,3);""" - createMV("create materialized view subQuery_mv as select deptno, empid from subQuery;") sleep(3000) @@ -60,6 +59,7 @@ suite ("subQuery") { */ sql """set enable_stats=true;""" + sql """alter table subQuery modify column time_col set stats ('row_count'='4');""" mv_rewrite_fail("select * from subQuery order by empid;", "subQuery_mv") } diff --git a/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy b/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy index 0f57b5c6c1687e..9feb1aa1c975f5 100644 --- a/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy +++ b/regression-test/suites/nereids_syntax_p0/mv/ut/unionDis.groovy @@ -58,6 +58,8 @@ suite ("unionDis") { order_qt_select_mv "select * from (select empid, deptno from unionDis where empid >1 union select empid, deptno from unionDis where empid <0) t order by 1;" sql """set enable_stats=true;""" + sql """alter table unionDis modify column time_col set stats ('row_count'='4');""" + mv_rewrite_fail("select * from unionDis order by empid;", "unionDis_mv") explain { diff --git a/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy b/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy index 0f682cacd8be96..3d65f3a85063d4 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/agg.groovy @@ -44,6 +44,7 @@ suite("agg") { AGGREGATE KEY (siteid,citycode,username) DISTRIBUTED BY HASH(siteid) BUCKETS 5 properties("replication_num" = "1"); """ + sql """alter table test_rollup_agg1 modify column siteid set stats ('row_count'='3');""" sql """ALTER TABLE ${tbName} ADD ROLLUP rollup_city(citycode, pv);""" int max_try_secs = 60 String res = "NOT_FINISHED" diff --git a/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy b/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy index b711728f769129..07af0da35098e1 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/agg_date.groovy @@ -44,6 +44,7 @@ suite("agg_date", "rollup") { AGGREGATE KEY (datek1, datetimek1, datetimek2, datetimek3) DISTRIBUTED BY HASH(datek1) BUCKETS 5 properties("replication_num" = "1"); """ + sql """alter table test_rollup_agg_date1 modify column datek1 set stats ('row_count'='2');""" sql """ALTER TABLE ${tbName} ADD ROLLUP rollup_date(datek1,datetimek2,datetimek1,datetimek3,datev1,datetimev1,datetimev2,datetimev3);""" int max_try_secs = 60 while (max_try_secs--) { diff --git a/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy b/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy index a3917dee28a75a..2407c70d8a451c 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/bitmap.groovy @@ -34,6 +34,7 @@ suite("bitmap", "rollup") { ) DISTRIBUTED BY HASH(k1) properties("replication_num" = "1"); """ + sql """alter table test_materialized_view_bitmap1 modify column k1 set stats ('row_count'='2');""" sql "CREATE MATERIALIZED VIEW test_neg as select k1,bitmap_union(to_bitmap(k2)), bitmap_union(to_bitmap(k3)) FROM ${tbName1} GROUP BY k1;" max_try_secs = 60 diff --git a/regression-test/suites/nereids_syntax_p0/rollup/date.groovy b/regression-test/suites/nereids_syntax_p0/rollup/date.groovy index 1415a442952ab6..138b8775778a6c 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/date.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/date.groovy @@ -61,6 +61,7 @@ suite("date", "rollup") { mv_rewrite_success("SELECT store_id, max(sale_datetime3) FROM ${tbName1} GROUP BY store_id", "amt_max4") sql """set enable_stats=true;""" + sql """alter table test_materialized_view_date1 modify column record_id set stats ('row_count'='2');""" mv_rewrite_success("SELECT store_id, max(sale_date1) FROM ${tbName1} GROUP BY store_id", "amt_max1") mv_rewrite_success("SELECT store_id, max(sale_datetime1) FROM ${tbName1} GROUP BY store_id", "amt_max2") diff --git a/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy b/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy index 2ba3540832945a..5002c509a20617 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/hll/hll.groovy @@ -46,6 +46,7 @@ suite("hll", "rollup") { "amt_count") sql """set enable_stats=true;""" + sql """alter table test_materialized_view_hll1 modify column record_id set stats ('row_count'='2');""" mv_rewrite_success("SELECT store_id, hll_union_agg(hll_hash(sale_amt)) FROM test_materialized_view_hll1 GROUP BY store_id;", "amt_count") } diff --git a/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy b/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy index 51599f294d1e62..d7dec7696d95eb 100644 --- a/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy +++ b/regression-test/suites/nereids_syntax_p0/rollup/hll_with_light_sc/hll_with_light_sc.groovy @@ -43,6 +43,7 @@ suite("hll_with_light_sc", "rollup") { "amt_count1") sql """set enable_stats=true;""" + sql """alter table test_materialized_view_hll_with_light_sc1 modify column record_id set stats ('row_count'='2');""" mv_rewrite_success("SELECT store_id, hll_union_agg(hll_hash(sale_amt)) FROM test_materialized_view_hll_with_light_sc1 GROUP BY store_id;", "amt_count1") } diff --git a/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy b/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy index 3971f304e38646..cf657cc8ef3e3d 100644 --- a/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy +++ b/regression-test/suites/query_p0/aggregate/aggregate_count1.groovy @@ -17,7 +17,7 @@ * under the License. */ -suite("aggregate_count1", "query,arrow_flight_sql") { +suite("aggregate_count1", "query") { sql """ DROP TABLE IF EXISTS aggregate_count1 """ sql """create table if not exists aggregate_count1 ( name varchar(128), diff --git a/regression-test/suites/query_p0/aggregate/array_agg.groovy b/regression-test/suites/query_p0/aggregate/array_agg.groovy index 217285b572c538..42fb3b131a4f86 100644 --- a/regression-test/suites/query_p0/aggregate/array_agg.groovy +++ b/regression-test/suites/query_p0/aggregate/array_agg.groovy @@ -277,8 +277,24 @@ suite("array_agg") { order_qt_sql_array_agg_map """ SELECT id, array_agg(km) FROM test_array_agg_complex GROUP BY id ORDER BY id """ order_qt_sql_array_agg_struct """ SELECT id, array_agg(ks) FROM test_array_agg_complex GROUP BY id ORDER BY id """ + + sql """ DROP TABLE IF EXISTS test_array_agg_ip;""" + sql """ + CREATE TABLE test_array_agg_ip( + k1 BIGINT , + k4 ipv4 , + k6 ipv6 , + s string + ) DISTRIBUTED BY HASH(k1) BUCKETS 1 PROPERTIES("replication_num" = "1"); + """ + sql """ insert into test_array_agg_ip values(1,123,34141,"0.0.0.123") , (2,3114,318903,"0.0.0.123") , (3,7832131,192837891738927931231,"2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D"),(4,null,null,"2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D"); """ + + + qt_select """select array_sort(array_agg(k4)),array_sort(array_agg(k6)) from test_array_agg_ip """ + sql "DROP TABLE `test_array_agg`" sql "DROP TABLE `test_array_agg1`" sql "DROP TABLE `test_array_agg_int`" sql "DROP TABLE `test_array_agg_decimal`" + sql "DROP TABLE `test_array_agg_ip`" } diff --git a/regression-test/suites/query_p0/aggregate/select_distinct.groovy b/regression-test/suites/query_p0/aggregate/select_distinct.groovy index 2d6a8679d87ed8..6456158bdadb0d 100644 --- a/regression-test/suites/query_p0/aggregate/select_distinct.groovy +++ b/regression-test/suites/query_p0/aggregate/select_distinct.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("select_distinct", "arrow_flight_sql") { +suite("select_distinct") { sql """DROP TABLE IF EXISTS decimal_a;""" sql """DROP TABLE IF EXISTS decimal_b;""" sql """DROP TABLE IF EXISTS decimal_c;""" diff --git a/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy b/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy index 8bd3b19088a486..45499fc6f248a2 100644 --- a/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy +++ b/regression-test/suites/query_p0/casesensetive_column/join_with_column_casesensetive.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("join_with_column_casesensetive", "arrow_flight_sql") { +suite("join_with_column_casesensetive") { def tables=["ad_order_data_v1","ad_order_data"] for (String table in tables) { diff --git a/regression-test/suites/query_p0/cast/test_cast.groovy b/regression-test/suites/query_p0/cast/test_cast.groovy index dae669e2965f04..947d61bc828861 100644 --- a/regression-test/suites/query_p0/cast/test_cast.groovy +++ b/regression-test/suites/query_p0/cast/test_cast.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('test_cast', "arrow_flight_sql") { +suite('test_cast') { def date = "date '2020-01-01'" def datev2 = "datev2 '2020-01-01'" def datetime = "timestamp '2020-01-01 12:34:45'" diff --git a/regression-test/suites/query_p0/except/test_query_except.groovy b/regression-test/suites/query_p0/except/test_query_except.groovy index 410e24f89b92de..1a2aa742d2910d 100644 --- a/regression-test/suites/query_p0/except/test_query_except.groovy +++ b/regression-test/suites/query_p0/except/test_query_except.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_query_except", "arrow_flight_sql") { +suite("test_query_except") { // test query except, depend on query_test_data_load.groovy sql "use test_query_db" qt_select_except1 """ diff --git a/regression-test/suites/query_p0/group_concat/test_group_concat.groovy b/regression-test/suites/query_p0/group_concat/test_group_concat.groovy index 522d66ed64b30b..5054dc2ee3a91d 100644 --- a/regression-test/suites/query_p0/group_concat/test_group_concat.groovy +++ b/regression-test/suites/query_p0/group_concat/test_group_concat.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_group_concat", "query,p0,arrow_flight_sql") { +suite("test_group_concat", "query,p0") { qt_select """ SELECT group_concat(k6) FROM test_query_db.test where k6='false' """ diff --git a/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy b/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy index f8180b0ab43846..1f12de6628a5eb 100644 --- a/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy +++ b/regression-test/suites/query_p0/grouping_sets/test_grouping_sets1.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_grouping_sets1", "arrow_flight_sql") { +suite("test_grouping_sets1") { qt_select """ select col1 diff --git a/regression-test/suites/query_p0/having/having.groovy b/regression-test/suites/query_p0/having/having.groovy index bbad236c973870..fb32b3834af4d6 100644 --- a/regression-test/suites/query_p0/having/having.groovy +++ b/regression-test/suites/query_p0/having/having.groovy @@ -19,7 +19,7 @@ // /testing/trino-product-tests/src/main/resources/sql-tests/testcases/aggregate // and modified by Doris. -suite("having", "query,p0,arrow_flight_sql") { +suite("having", "query,p0") { sql """DROP TABLE IF EXISTS supplier""" sql """CREATE TABLE `supplier` ( `s_suppkey` int(11) NOT NULL, diff --git a/regression-test/suites/query_p0/intersect/test_intersect.groovy b/regression-test/suites/query_p0/intersect/test_intersect.groovy index 7919bec324b876..1c007b95d7d07d 100644 --- a/regression-test/suites/query_p0/intersect/test_intersect.groovy +++ b/regression-test/suites/query_p0/intersect/test_intersect.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_intersect", "arrow_flight_sql") { +suite("test_intersect") { qt_select """ SELECT * FROM (SELECT k1 FROM test_query_db.baseall INTERSECT SELECT k1 FROM test_query_db.test) a ORDER BY k1 diff --git a/regression-test/suites/query_p0/join/test_join2.groovy b/regression-test/suites/query_p0/join/test_join2.groovy index 9158133948f754..6125b9a873f77e 100644 --- a/regression-test/suites/query_p0/join/test_join2.groovy +++ b/regression-test/suites/query_p0/join/test_join2.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_join2", "query,p0,arrow_flight_sql") { +suite("test_join2", "query,p0") { def DBname = "regression_test_join2" def TBname1 = "J1_TBL" def TBname2 = "J2_TBL" diff --git a/regression-test/suites/query_p0/join/test_left_join1.groovy b/regression-test/suites/query_p0/join/test_left_join1.groovy index 104adab4a850d0..d4cbeeee65eda2 100644 --- a/regression-test/suites/query_p0/join/test_left_join1.groovy +++ b/regression-test/suites/query_p0/join/test_left_join1.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_left_join1", "query,p0,arrow_flight_sql") { +suite("test_left_join1", "query,p0") { def tableName = "test_left_join1" sql """drop table if exists ${tableName}""" diff --git a/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy b/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy index f99dfa042446e9..ad19e554690ee7 100644 --- a/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy +++ b/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_nestedloop_outer_join", "query_p0,arrow_flight_sql") { +suite("test_nestedloop_outer_join", "query_p0") { def tbl1 = "test_nestedloop_outer_join1" def tbl2 = "test_nestedloop_outer_join2" 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 676cdd06274a68..cbe09ec527ffbc 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 @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_partitioned_hash_join", "query,p0,arrow_flight_sql") { +suite("test_partitioned_hash_join", "query,p0") { sql "drop table if exists test_partitioned_hash_join_l" sql "drop table if exists test_partitioned_hash_join_r" sql """ create table test_partitioned_hash_join_l ( diff --git a/regression-test/suites/query_p0/lateral_view/lateral_view.groovy b/regression-test/suites/query_p0/lateral_view/lateral_view.groovy index bfe6ca76872ea8..a24623590cd0af 100644 --- a/regression-test/suites/query_p0/lateral_view/lateral_view.groovy +++ b/regression-test/suites/query_p0/lateral_view/lateral_view.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("lateral_view", "arrow_flight_sql") { +suite("lateral_view") { sql """ DROP TABLE IF EXISTS `test_explode_bitmap` """ sql """ CREATE TABLE `test_explode_bitmap` ( diff --git a/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy b/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy index caa75ac7be38ed..da0c7231f425d1 100644 --- a/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy +++ b/regression-test/suites/query_p0/limit/OffsetInSubqueryWithJoin.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_offset_in_subquery_with_join", "query,arrow_flight_sql") { +suite("test_offset_in_subquery_with_join", "query") { // define a sql table def testTable = "test_offset_in_subquery_with_join" diff --git a/regression-test/suites/query_p0/literal_view/lietral_test.groovy b/regression-test/suites/query_p0/literal_view/lietral_test.groovy index 27b82c16247a73..6e9d51f0a0d610 100644 --- a/regression-test/suites/query_p0/literal_view/lietral_test.groovy +++ b/regression-test/suites/query_p0/literal_view/lietral_test.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("literal_view_test", "arrow_flight_sql") { +suite("literal_view_test") { sql """DROP TABLE IF EXISTS table1""" diff --git a/regression-test/suites/query_p0/operator/test_set_operator.groovy b/regression-test/suites/query_p0/operator/test_set_operator.groovy index 7d6219585e4c4c..cb05e18b3e870b 100644 --- a/regression-test/suites/query_p0/operator/test_set_operator.groovy +++ b/regression-test/suites/query_p0/operator/test_set_operator.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_set_operators", "query,p0,arrow_flight_sql") { +suite("test_set_operators", "query,p0") { sql """ DROP TABLE IF EXISTS t1; diff --git a/regression-test/suites/query_p0/operator/test_sort_operator.groovy b/regression-test/suites/query_p0/operator/test_sort_operator.groovy index d76daff01f6fcc..24a2b8ef73a424 100644 --- a/regression-test/suites/query_p0/operator/test_sort_operator.groovy +++ b/regression-test/suites/query_p0/operator/test_sort_operator.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_sort_operator", "query,p0,arrow_flight_sql") { +suite("test_sort_operator", "query,p0") { sql """ DROP TABLE IF EXISTS dim_org_ful; diff --git a/regression-test/suites/query_p0/session_variable/test_default_limit.groovy b/regression-test/suites/query_p0/session_variable/test_default_limit.groovy index 2ce3b647142ae4..edda5d51790c56 100644 --- a/regression-test/suites/query_p0/session_variable/test_default_limit.groovy +++ b/regression-test/suites/query_p0/session_variable/test_default_limit.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('test_default_limit', "arrow_flight_sql") { +suite('test_default_limit') { sql 'drop table if exists baseall' sql 'drop table if exists bigtable' diff --git a/regression-test/suites/query_p0/show/test_show_create_table.groovy b/regression-test/suites/query_p0/show/test_show_create_table.groovy index 1e3fc7ff5cb527..6325cbe319fd88 100644 --- a/regression-test/suites/query_p0/show/test_show_create_table.groovy +++ b/regression-test/suites/query_p0/show/test_show_create_table.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_show_create_table", "query,arrow_flight_sql") { +suite("test_show_create_table", "query") { String tb_name = "tb_show_create_table"; try { sql """drop table if exists ${tb_name} """ diff --git a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy index cdab9472e27dbd..86a951d7ac33c6 100644 --- a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_aggregate_all_functions", "arrow_flight_sql") { +suite("test_aggregate_all_functions") { sql "set batch_size = 4096" diff --git a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy index 5138db6e73b4ad..269a0bf0db87cf 100644 --- a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy +++ b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function_null.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_case_function_null", "query,p0,arrow_flight_sql") { +suite("test_case_function_null", "query,p0") { sql """ drop table if exists case_null0 """ sql """ create table case_null0 ( `c0` decimalv3(17, 1) NULL, diff --git a/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy b/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy index d547e9fb287d71..590ccd10821f61 100644 --- a/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/hash_functions/test_hash_function.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_hash_function", "arrow_flight_sql") { +suite("test_hash_function") { sql "set batch_size = 4096;" sql "set enable_profile = true;" diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy index 5373217503a018..03e9788a58a3b8 100644 --- a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy +++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_ip_functions", "arrow_flight_sql") { +suite("test_ip_functions") { sql "set batch_size = 4096;" qt_sql "SELECT ipv4_num_to_string(-1);" diff --git a/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy b/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy index b5865034538a11..c885e3ae3431f3 100644 --- a/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy +++ b/regression-test/suites/query_p0/sql_functions/json_function/test_query_json_insert.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_query_json_insert", "query,arrow_flight_sql") { +suite("test_query_json_insert", "query") { qt_sql "select json_insert('{\"a\": 1, \"b\": [2, 3]}', '\$', null);" qt_sql "select json_insert('{\"k\": [1, 2]}', '\$.k[0]', null, '\$.[1]', null);" def tableName = "test_query_json_insert" diff --git a/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy b/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy index 4bd88bf131e727..aa0deec96f46a2 100644 --- a/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/json_functions/test_json_function.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_json_function", "arrow_flight_sql") { +suite("test_json_function") { sql "set batch_size = 4096;" qt_sql "SELECT get_json_double('{\"k1\":1.3, \"k2\":\"2\"}', \"\$.k1\");" diff --git a/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy b/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy index 3a74abfe9c8b22..6c4867174d11ac 100644 --- a/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy +++ b/regression-test/suites/query_p0/sql_functions/math_functions/test_conv.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_conv", "arrow_flight_sql") { +suite("test_conv") { qt_select "SELECT CONV(15,10,2)" sql """ drop table if exists test_conv; """ diff --git a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy index f1487d283dfcdf..061665d3b9da6e 100644 --- a/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy +++ b/regression-test/suites/query_p0/sql_functions/search_functions/test_multi_string_search.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_multi_string_search", "arrow_flight_sql") { +suite("test_multi_string_search") { def table_name = "test_multi_string_search_strings" sql """ DROP TABLE IF EXISTS ${table_name} """ diff --git a/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy b/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy index f76cb44cb4ad4b..e98e11ba7e6888 100644 --- a/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/spatial_functions/test_gis_function.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_gis_function", "arrow_flight_sql") { +suite("test_gis_function") { sql "set batch_size = 4096;" qt_sql "SELECT ST_AsText(ST_Point(24.7, 56.7));" diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy index 6e18fb57eeb4cf..f5d32653c818b5 100644 --- a/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_string_function.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_string_function", "arrow_flight_sql") { +suite("test_string_function") { sql "set batch_size = 4096;" qt_sql "select elt(0, \"hello\", \"doris\");" diff --git a/regression-test/suites/query_p0/sql_functions/string_functions/test_trim_in.groovy b/regression-test/suites/query_p0/sql_functions/string_functions/test_trim_in.groovy new file mode 100644 index 00000000000000..ae6790fb0693e1 --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/string_functions/test_trim_in.groovy @@ -0,0 +1,204 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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_trim_in") { + // this table has nothing todo. just make it eaiser to generate query + sql " drop table if exists hits_two_args " + sql """ create table hits_two_args( + nothing boolean + ) + properties("replication_num" = "1"); + """ + sql "insert into hits_two_args values(true);" + + sql " drop table if exists test_trim_in" + sql """ + create table test_trim_in ( + k0 int, + a varchar not null, + b varchar null + ) + DISTRIBUTED BY HASH(k0) + PROPERTIES + ( + "replication_num" = "1" + ); + """ + order_qt_empty_nullable "select trim_in(a, 'x') from test_trim_in" + order_qt_empty_not_nullable "select trim_in(b, 'x') from test_trim_in" + + sql "insert into test_trim_in values (1, '1', null), (1, '1', null), (1, '1', null)" + order_qt_all_null "select trim_in(b, NULL) from test_trim_in" + + sql "truncate table test_trim_in" + sql """ insert into test_trim_in values (1, "", ""), (2, "abcd", "ac"), (3, ' hello ', 'he '), + (4, ' hello world ', ' ehlowrd'),(5, ' hello world ', ' eh'),(6,'浙江省杭州市','余杭区'),(6,'c浙江省杭州市a','西湖区'); + """ + + /// all values + order_qt_nullable """ + SELECT atan2(t.arg1_two_args, t.ARG2) as result + FROM ( + SELECT hits_two_args.nothing, TABLE1.arg1_two_args, TABLE1.order1, TABLE2.ARG2, TABLE2.order2 + FROM hits_two_args + CROSS JOIN ( + SELECT b as arg1_two_args, k0 as order1 + FROM test_trim_in + ) as TABLE1 + CROSS JOIN ( + SELECT b as ARG2, k0 as order2 + FROM test_trim_in + ) as TABLE2 + )t; + """ + + /// nullables + order_qt_not_nullable "select trim_in(a, 'he') from test_trim_in" + order_qt_nullable "select trim_in(b, 'he') from test_trim_in" + order_qt_not_nullable_null "select trim_in(a, NULL) from test_trim_in" + order_qt_nullable_null "select trim_in(b, NULL) from test_trim_in" + + /// consts. most by BE-UT + order_qt_const_nullable "select trim_in(NULL,NULL) from test_trim_in" + order_qt_partial_const_nullable "select trim_in(NULL, 'he') from test_trim_in" + order_qt_const_partial_nullable_no_null "select trim_in(nullable('abcd'), 'cde') from test_trim_in" + order_qt_const_other_not_nullable "select trim_in(a, 'x') from test_trim_in" + order_qt_const_not_nullable "select trim_in('abdc', 'df') from test_trim_in" + + + + /// folding + def re_fe + def re_be + def re_no_fold + def check_three_ways = { test_sql -> + re_fe = order_sql "select/*+SET_VAR(enable_fold_constant_by_be=false)*/ ${test_sql}" + re_be = order_sql "select/*+SET_VAR(enable_fold_constant_by_be=true)*/ ${test_sql}" + re_no_fold = order_sql "select/*+SET_VAR(debug_skip_fold_constant=true)*/ ${test_sql}" + logger.info("check on sql ${test_sql}") + assertEquals(re_fe, re_be) + assertEquals(re_fe, re_no_fold) + } + + check_three_ways "trim_in(' hello world ', ' ld')" + check_three_ways "ltrim_in(' hello world ', ' ld')" + check_three_ways "rtrim_in(' hello world ', ' ld')" + check_three_ways "trim_in(' hello world ', ' ehlowrd')" + check_three_ways "ltrim_in(' hello world ', ' ehlowrd')" + check_three_ways "rtrim_in(' hello world ', ' ehlowrd')" + check_three_ways "trim_in(' hello world ', '')" + check_three_ways "ltrim_in(' hello world ', '')" + check_three_ways "rtrim_in(' hello world ', '')" + check_three_ways "trim_in(' hello world ', ' ')" + check_three_ways "ltrim_in(' hello world ', ' ')" + check_three_ways "rtrim_in(' hello world ', ' ')" + + order_qt_1 "SELECT ltrim_in('');" + order_qt_2 "SELECT ltrim_in(' ');" + order_qt_3 "SELECT ltrim_in(' hello ');" + order_qt_4 "SELECT ltrim_in(' hello');" + order_qt_5 "SELECT ltrim_in('hello ');" + order_qt_6 "SELECT ltrim_in(' hello world ');" + order_qt_7 "SELECT ltrim_in(CAST('' AS CHAR(20)));" + order_qt_8 "SELECT ltrim_in(CAST(' hello ' AS CHAR(9)));" + order_qt_9 "SELECT ltrim_in(CAST(' hello' AS CHAR(7)));" + order_qt_10 "SELECT ltrim_in(CAST('hello ' AS CHAR(7)));" + order_qt_11 "SELECT ltrim_in(CAST(' hello world ' AS CHAR(13)));" + order_qt_12 "SELECT rtrim_in('');" + order_qt_13 "SELECT rtrim_in(' ');" + order_qt_14 "SELECT rtrim_in(' hello ');" + order_qt_15 "SELECT rtrim_in(' hello');" + order_qt_16 "SELECT rtrim_in('hello ');" + order_qt_17 "SELECT rtrim_in(' hello world ');" + order_qt_18 "SELECT rtrim_in(CAST('' AS CHAR(20)));" + order_qt_19 "SELECT rtrim_in(CAST(' hello ' AS CHAR(9)));" + order_qt_20 "SELECT rtrim_in(CAST(' hello' AS CHAR(7)));" + order_qt_21 "SELECT rtrim_in(CAST('hello ' AS CHAR(7)));" + order_qt_22 "SELECT rtrim_in(CAST(' hello world ' AS CHAR(13)));" + order_qt_23 "SELECT ltrim_in('', '');" + order_qt_24 "SELECT ltrim_in(' ', '');" + order_qt_25 "SELECT ltrim_in(' hello ', '');" + order_qt_26 "SELECT ltrim_in(' hello ', ' ');" + order_qt_27 "SELECT ltrim_in(' hello ', ' ');" + order_qt_28 "SELECT ltrim_in(' hello ', 'he ');" + order_qt_29 "SELECT ltrim_in(' hello', ' ');" + order_qt_30 "SELECT ltrim_in(' hello', 'e h');" + order_qt_31 "SELECT ltrim_in('hello ', 'l');" + order_qt_32 "SELECT ltrim_in(' hello world ', ' ');" + order_qt_33 "SELECT ltrim_in(' hello world ', ' eh');" + order_qt_34 "SELECT ltrim_in(' hello world ', ' ehlowrd');" + order_qt_35 "SELECT ltrim_in(' hello world ', ' x');" + order_qt_36 "SELECT ltrim_in(CAST('' AS CHAR(1)), '');" + order_qt_37 "SELECT ltrim_in(CAST(' ' AS CHAR(3)), '');" + order_qt_38 "SELECT ltrim_in(CAST(' hello ' AS CHAR(9)), '');" + order_qt_39 "SELECT ltrim_in(CAST(' hello ' AS CHAR(9)), ' ');" + order_qt_40 "SELECT ltrim_in(CAST(' hello ' AS CHAR(9)), 'he ');" + order_qt_41 "SELECT ltrim_in(CAST(' hello' AS CHAR(7)), ' ');" + order_qt_42 "SELECT ltrim_in(CAST(' hello' AS CHAR(7)), 'e h');" + order_qt_43 "SELECT ltrim_in(CAST('hello ' AS CHAR(7)), 'l');" + order_qt_44 "SELECT ltrim_in(CAST(' hello world ' AS CHAR(13)), ' ');" + order_qt_45 "SELECT ltrim_in(CAST(' hello world ' AS CHAR(13)), ' eh');" + order_qt_46 "SELECT ltrim_in(CAST(' hello world ' AS CHAR(13)), ' ehlowrd');" + order_qt_47 "SELECT ltrim_in(CAST(' hello world ' AS CHAR(13)), ' x');" + order_qt_48 "SELECT rtrim_in('', '');" + order_qt_49 "SELECT rtrim_in(' ', '');" + order_qt_50 "SELECT rtrim_in(' hello ', '');" + order_qt_51 "SELECT rtrim_in(' hello ', ' ');" + order_qt_52 "SELECT rtrim_in(' hello ', 'lo ');" + order_qt_53 "SELECT rtrim_in('hello ', ' ');" + order_qt_54 "SELECT rtrim_in('hello ', 'l o');" + order_qt_55 "SELECT rtrim_in('hello ', 'l');" + order_qt_56 "SELECT rtrim_in(' hello world ', ' ');" + order_qt_57 "SELECT rtrim_in(' hello world ', ' ld');" + order_qt_58 "SELECT rtrim_in(' hello world ', ' ehlowrd');" + order_qt_59 "SELECT rtrim_in(' hello world ', ' x');" + order_qt_60 "SELECT rtrim_in(CAST('abc def' AS CHAR(7)), 'def');" + order_qt_61 "SELECT rtrim_in(CAST('' AS CHAR(1)), '');" + order_qt_62 "SELECT rtrim_in(CAST(' ' AS CHAR(3)), '');" + order_qt_63 "SELECT rtrim_in(CAST(' hello ' AS CHAR(9)), '');" + order_qt_64 "SELECT rtrim_in(CAST(' hello ' AS CHAR(9)), ' ');" + order_qt_65 "SELECT rtrim_in(CAST(' hello ' AS CHAR(9)), 'he ');" + order_qt_66 "SELECT rtrim_in(CAST(' hello' AS CHAR(7)), ' ');" + order_qt_67 "SELECT rtrim_in(CAST(' hello' AS CHAR(7)), 'e h');" + order_qt_68 "SELECT rtrim_in(CAST('hello ' AS CHAR(7)), 'l');" + order_qt_69 "SELECT rtrim_in(CAST(' hello world ' AS CHAR(13)), ' ');" + order_qt_70 "SELECT rtrim_in(CAST(' hello world ' AS CHAR(13)), ' eh');" + order_qt_71 "SELECT rtrim_in(CAST(' hello world ' AS CHAR(13)), ' ehlowrd');" + order_qt_72 "SELECT rtrim_in(CAST(' hello world ' AS CHAR(13)), ' x');" + + order_qt_73 "SELECT trim_in(a, ' eh') from test_trim_in;" + order_qt_74 "SELECT ltrim_in(a, ' eh') from test_trim_in;" + order_qt_75 "SELECT rtrim_in(a, ' eh') from test_trim_in;" + order_qt_76 "SELECT trim_in(b, ' eh') from test_trim_in;" + order_qt_77 "SELECT ltrim_in(b, ' eh') from test_trim_in;" + order_qt_78 "SELECT rtrim_in(b, ' eh') from test_trim_in;" + order_qt_79 "SELECT trim_in(a, NULL) from test_trim_in;" + order_qt_80 "SELECT ltrim_in(a, NULL) from test_trim_in;" + order_qt_81 "SELECT rtrim_in(a, NULL) from test_trim_in;" + order_qt_82 "SELECT trim_in(b, NULL) from test_trim_in;" + order_qt_83 "SELECT ltrim_in(b, NULL) from test_trim_in;" + order_qt_84 "SELECT rtrim_in(b, NULL) from test_trim_in;" + order_qt_85 "SELECT trim_in(a, '省市杭州') from test_trim_in;" + order_qt_86 "SELECT ltrim_in(a, '省市杭州') from test_trim_in;" + order_qt_87 "SELECT rtrim_in(a, '省市杭州') from test_trim_in;" + order_qt_88 "SELECT trim_in(b, '杭余') from test_trim_in;" + order_qt_89 "SELECT ltrim_in(b, '杭余') from test_trim_in;" + order_qt_90 "SELECT rtrim_in(b, '杭余') from test_trim_in;" + order_qt_91 "SELECT trim_in(a, '省市a杭州') from test_trim_in;" + order_qt_92 "SELECT ltrim_in(a, '省市b杭州') from test_trim_in;" + order_qt_93 "SELECT rtrim_in(a, '省市c杭州') from test_trim_in;" +} diff --git a/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy b/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy index 53db931c03bb03..b7dd4d640799fb 100644 --- a/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy +++ b/regression-test/suites/query_p0/sql_functions/table_function/explode_split.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("explode_split", "arrow_flight_sql") { +suite("explode_split") { def tableName = "test_lv_str" sql """ DROP TABLE IF EXISTS ${tableName} """ diff --git a/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy b/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy index 095ec89e220f1b..8e0e94fa2df805 100644 --- a/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/test_alias_function.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('test_alias_function', "arrow_flight_sql") { +suite('test_alias_function') { sql ''' CREATE ALIAS FUNCTION IF NOT EXISTS f1(DATETIMEV2(3), INT) with PARAMETER (datetime1, int1) as date_trunc(days_sub(datetime1, int1), 'day')''' diff --git a/regression-test/suites/query_p0/sql_functions/test_predicate.groovy b/regression-test/suites/query_p0/sql_functions/test_predicate.groovy index 6cca6b62c9960b..20b3c179ad5c01 100644 --- a/regression-test/suites/query_p0/sql_functions/test_predicate.groovy +++ b/regression-test/suites/query_p0/sql_functions/test_predicate.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_predicate", "arrow_flight_sql") { +suite("test_predicate") { sql """drop table if exists t1;""" sql """ create table t1 ( diff --git a/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy b/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy index 1a455da92446f8..d0862a580ca600 100644 --- a/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/width_bucket_fuctions/test_width_bucket_function.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_width_bucket_function", "arrow_flight_sql") { +suite("test_width_bucket_function") { qt_sql "select width_bucket(1, 2, 3, 2)" qt_sql "select width_bucket(null, 2, 3, 2)" qt_sql "select width_bucket(6, 2, 6, 4)" diff --git a/regression-test/suites/query_p0/subquery/test_subquery2.groovy b/regression-test/suites/query_p0/subquery/test_subquery2.groovy index a14a44fa152b97..e572459cc72fe3 100644 --- a/regression-test/suites/query_p0/subquery/test_subquery2.groovy +++ b/regression-test/suites/query_p0/subquery/test_subquery2.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_subquery2", "arrow_flight_sql") { +suite("test_subquery2") { sql """DROP TABLE IF EXISTS subquerytest2""" diff --git a/regression-test/suites/query_p0/test_data_type_marks.groovy b/regression-test/suites/query_p0/test_data_type_marks.groovy index 51fb7c9614e488..79803d98723313 100644 --- a/regression-test/suites/query_p0/test_data_type_marks.groovy +++ b/regression-test/suites/query_p0/test_data_type_marks.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_data_type_marks", "arrow_flight_sql") { +suite("test_data_type_marks") { def tbName = "org" sql "DROP TABLE IF EXISTS ${tbName}" sql """ diff --git a/regression-test/suites/query_p0/test_dict_with_null.groovy b/regression-test/suites/query_p0/test_dict_with_null.groovy index 83d253fa4d1b04..b3738bb68aa1ba 100644 --- a/regression-test/suites/query_p0/test_dict_with_null.groovy +++ b/regression-test/suites/query_p0/test_dict_with_null.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("dict_with_null", "query,arrow_flight_sql") { +suite("dict_with_null", "query") { def tableName = "test_dict_with_null" sql "DROP TABLE IF EXISTS ${tableName}" sql """ diff --git a/regression-test/suites/query_p0/test_orderby_nullliteral.groovy b/regression-test/suites/query_p0/test_orderby_nullliteral.groovy index e806060c8bcb1c..fe11c778af0b98 100644 --- a/regression-test/suites/query_p0/test_orderby_nullliteral.groovy +++ b/regression-test/suites/query_p0/test_orderby_nullliteral.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("orderby_nullliteral", "query,arrow_flight_sql") { +suite("orderby_nullliteral", "query") { def tableName = "test_orderby_nullliteral" sql "DROP TABLE IF EXISTS ${tableName}" diff --git a/regression-test/suites/query_p0/test_select_constant.groovy b/regression-test/suites/query_p0/test_select_constant.groovy index 68f0a28a20e853..6015e19576c690 100644 --- a/regression-test/suites/query_p0/test_select_constant.groovy +++ b/regression-test/suites/query_p0/test_select_constant.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_select_constant", "arrow_flight_sql") { +suite("test_select_constant") { qt_select1 'select 100, "test", date("2021-01-02");' qt_select_geo1 'SELECT ST_AsText(ST_Point(123.12345678901234567890,89.1234567890));' } diff --git a/regression-test/suites/query_p0/test_select_with_predicate_like.groovy b/regression-test/suites/query_p0/test_select_with_predicate_like.groovy index 0d01f1b958a11c..9491c4271ca530 100644 --- a/regression-test/suites/query_p0/test_select_with_predicate_like.groovy +++ b/regression-test/suites/query_p0/test_select_with_predicate_like.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_select_with_predicate_like", "arrow_flight_sql") { +suite("test_select_with_predicate_like") { def tables=["test_basic_agg"] for (String table in tables) { diff --git a/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy b/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy index ccd1b9160fb148..768e04b4c327b5 100644 --- a/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy +++ b/regression-test/suites/query_p0/test_select_with_predicate_prune.groovy @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -suite("test_select_with_predicate_prune", "arrow_flight_sql") { +suite("test_select_with_predicate_prune") { sql """ drop table if exists `test_select_with_predicate_prune`; """ diff --git a/regression-test/suites/query_p0/type_inference/test_largeint.groovy b/regression-test/suites/query_p0/type_inference/test_largeint.groovy index 161359cfa97e72..d5cbfa4b479838 100644 --- a/regression-test/suites/query_p0/type_inference/test_largeint.groovy +++ b/regression-test/suites/query_p0/type_inference/test_largeint.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_largeint", "arrow_flight_sql") { +suite("test_largeint") { def tbName = "test_largeint" sql "DROP TABLE IF EXISTS ${tbName}" sql """ diff --git a/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy b/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy index d563ef1630517d..99164a999c557e 100644 --- a/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy +++ b/regression-test/suites/query_p0/with/test_with_and_two_phase_agg.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_with_and_two_phase_agg", "arrow_flight_sql") { +suite("test_with_and_two_phase_agg") { def tableName = "test_with_and_two_phase_agg_table" sql """ DROP TABLE IF EXISTS ${tableName} """ sql """ diff --git a/regression-test/suites/statistics/analyze_stats.groovy b/regression-test/suites/statistics/analyze_stats.groovy index 55074e995fe68d..69360da6911ba5 100644 --- a/regression-test/suites/statistics/analyze_stats.groovy +++ b/regression-test/suites/statistics/analyze_stats.groovy @@ -2903,7 +2903,38 @@ PARTITION `p599` VALUES IN (599) assertEquals("521779.0", alter_result[0][5]) assertEquals("7.142863009760572", alter_result[0][6]) - sql """DROP DATABASE IF EXISTS trigger""" + + // Test show last analyze table version + sql """create database if not exists test_version""" + sql """use test_version""" + sql """drop table if exists region""" + sql """ + CREATE TABLE region ( + r_regionkey int NOT NULL, + r_name VARCHAR(25) NOT NULL, + r_comment VARCHAR(152) + )ENGINE=OLAP + DUPLICATE KEY(`r_regionkey`) + COMMENT "OLAP" + DISTRIBUTED BY HASH(`r_regionkey`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """analyze table region with sync""" + def versionResult = sql """show column stats region""" + assertEquals(versionResult[0][16], "1") + assertEquals(versionResult[1][16], "1") + assertEquals(versionResult[2][16], "1") + + sql """insert into region values (1, "1", "1")""" + sql """analyze table region with sync""" + versionResult = sql """show column stats region""" + assertEquals(versionResult[0][16], "2") + assertEquals(versionResult[1][16], "2") + assertEquals(versionResult[2][16], "2") + + sql """drop database if exists test_version""" } diff --git a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy index 1ec60fbb10e837..d689deeb8afe02 100644 --- a/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy +++ b/regression-test/suites/unique_with_mow_p0/partial_update/test_partial_update_auto_inc.groovy @@ -103,6 +103,46 @@ suite("test_partial_update_auto_inc") { time 10000 } order_qt_select_6 "select * from test_primary_key_partial_update_auto_inc2" - sql """ DROP TABLE IF EXISTS test_primary_key_partial_update_auto_inc2 """ + + + sql """ DROP TABLE IF EXISTS test_primary_key_partial_update_auto_inc3 force; """ + sql """ create table test_primary_key_partial_update_auto_inc3 + ( + `id` bigint not null AUTO_INCREMENT, + `project_code` varchar(20) not null, + `period_num` int, + `c2` int + ) unique KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS auto + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "enable_unique_key_merge_on_write" = "true" + ); """ + sql "set enable_unique_key_partial_update=true;" + sql "set enable_insert_strict=false;" + sql "sync;" + + sql "insert into test_primary_key_partial_update_auto_inc3(project_code,period_num) values ('test1',15),('test2',29),('test3',49);" + qt_sql "select project_code,period_num from test_primary_key_partial_update_auto_inc3 order by project_code,period_num;" + qt_sql "select count(distinct id) from test_primary_key_partial_update_auto_inc3;" + + + sql """ DROP TABLE IF EXISTS test_primary_key_partial_update_auto_inc4 """ + sql """ CREATE TABLE test_primary_key_partial_update_auto_inc4 ( + `k1` BIGINT NOT NULL AUTO_INCREMENT, + `k2` int, + `c1` int, + `c2` int, + `c3` int) + UNIQUE KEY(`k1`,`k2`) DISTRIBUTED BY HASH(`k1`,`k2`) BUCKETS 1 + PROPERTIES("replication_num" = "1", "enable_unique_key_merge_on_write" = "true"); """ + sql "set enable_unique_key_partial_update=true;" + sql "set enable_insert_strict=false;" + sql "sync;" + + test { + sql "insert into test_primary_key_partial_update_auto_inc4(c1,c2) values(1,1),(2,2),(3,3)" + exception "Partial update should include all key columns, missing: k2" + } } } diff --git a/regression-test/suites/variant_github_events_new_p2/sql/affinityByIssuesAndPRs1.sql b/regression-test/suites/variant_github_events_new_p2/sql/affinityByIssuesAndPRs1.sql deleted file mode 100644 index 043ff99f6a0ec4..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/affinityByIssuesAndPRs1.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS prs, - count(distinct cast(v["actor"]["login"] as string)) AS authors -FROM github_events -WHERE (cast(v["type"] as string) = 'PullRequestEvent') AND (cast(v["payload"]["action"] as string) = 'opened') AND (cast(v["actor"]["login"] as string) IN -( - SELECT cast(v["actor"]["login"] as string) - FROM github_events - WHERE (cast(v["type"] as string) = 'PullRequestEvent') AND (cast(v["payload"]["action"] as string)= 'opened') AND (cast(v["repo"]["name"] as string) IN ('rspec/rspec-core', 'golden-warning/giraffedraft-server', 'apache/spark')) -)) AND (lower(cast(v["repo"]["name"] as string)) NOT LIKE '%clickhouse%') -GROUP BY repo_name -ORDER BY authors DESC, prs DESC, repo_name DESC -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/affinityByIssuesAndPRs2.sql b/regression-test/suites/variant_github_events_new_p2/sql/affinityByIssuesAndPRs2.sql deleted file mode 100644 index b3cf37124fc802..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/affinityByIssuesAndPRs2.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS prs, - count(distinct cast(v["actor"]["login"] as string)) AS authors -FROM github_events -WHERE (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') AND (cast(v["actor"]["login"] as string) IN -( - SELECT cast(v["actor"]["login"] as string) - FROM github_events - WHERE (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') AND (cast(v["repo"]["name"] as string) IN ('No-CQRT/GooGuns', 'ivolunteerph/ivolunteerph', 'Tribler/tribler')) -)) AND (lower(cast(v["repo"]["name"] as string)) NOT LIKE '%clickhouse%') -GROUP BY repo_name -ORDER BY authors DESC, prs DESC, repo_name ASC -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/authorsWithTheMostPushes.sql b/regression-test/suites/variant_github_events_new_p2/sql/authorsWithTheMostPushes.sql deleted file mode 100644 index 637dba7f6206e8..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/authorsWithTheMostPushes.sql +++ /dev/null @@ -1,9 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["actor"]["login"] as string) as actor_login, - count() AS c, - count(distinct cast(v["repo"]["name"] as string)) AS repos - FROM github_events - WHERE cast(v["type"] as string) = 'PushEvent' - GROUP BY actor_login - ORDER BY c DESC, 1, 3 - LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/countingStar1.sql b/regression-test/suites/variant_github_events_new_p2/sql/countingStar1.sql deleted file mode 100644 index cae7d585bf0784..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/countingStar1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count() FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' diff --git a/regression-test/suites/variant_github_events_new_p2/sql/countingStar2.sql b/regression-test/suites/variant_github_events_new_p2/sql/countingStar2.sql deleted file mode 100644 index 15bc6efe16961b..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/countingStar2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["payload"]["action"] as string) as payload_action, count() FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY payload_action \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/countingStar3.sql b/regression-test/suites/variant_github_events_new_p2/sql/countingStar3.sql deleted file mode 100644 index 172967acbfdd24..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/countingStar3.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count() FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND cast(v["repo"]["name"] as string) IN ('apache/spark', 'GunZi200/Memory-Colour', 'isohuntto/openbay', 'wasabeef/awesome-android-ui') GROUP BY cast(v["payload"]["action"] as string) \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/distributionOfRepositoriesByStarCount.sql b/regression-test/suites/variant_github_events_new_p2/sql/distributionOfRepositoriesByStarCount.sql deleted file mode 100644 index 5dbec453df93ef..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/distributionOfRepositoriesByStarCount.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - pow(10, floor(log10(c))) AS stars, - count(distinct repo) -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo, - count() AS c - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' - GROUP BY repo -) t -GROUP BY stars -ORDER BY stars ASC diff --git a/regression-test/suites/variant_github_events_new_p2/sql/githubRoulette.sql b/regression-test/suites/variant_github_events_new_p2/sql/githubRoulette.sql deleted file mode 100644 index 20cd59b11ecccc..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/githubRoulette.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' ORDER BY cast(v["created_at"] as datetime), cast(v["repo"]["name"] as string) LIMIT 50 \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql deleted file mode 100644 index e3f1053a6be1c2..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND year(cast(v["created_at"] as datetime)) = '2015' GROUP BY repo_name ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql deleted file mode 100644 index e3f1053a6be1c2..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND year(cast(v["created_at"] as datetime)) = '2015' GROUP BY repo_name ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql deleted file mode 100644 index f68bdcfbf0e57a..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND year(cast(v["created_at"] as datetime)) = '2015' GROUP BY repo_name ORDER BY stars, repo_name DESC LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql deleted file mode 100644 index e3f1053a6be1c2..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND year(cast(v["created_at"] as datetime)) = '2015' GROUP BY repo_name ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql deleted file mode 100644 index e3f1053a6be1c2..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND year(cast(v["created_at"] as datetime)) = '2015' GROUP BY repo_name ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql deleted file mode 100644 index e3f1053a6be1c2..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND year(cast(v["created_at"] as datetime)) = '2015' GROUP BY repo_name ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql deleted file mode 100644 index 8f7e3f7b36fbfa..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql +++ /dev/null @@ -1,30 +0,0 @@ --- FIXME: UNSTABLE --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- repo, --- year, --- cnt --- FROM --- ( --- SELECT --- row_number() OVER (PARTITION BY year ORDER BY cnt DESC) AS r, --- repo, --- year, --- cnt --- FROM --- ( --- SELECT --- lower(cast(v["repo"]["name"] as string)) AS repo, --- year(cast(v["created_at"] as datetime)) AS year, --- count() AS cnt --- FROM github_events --- WHERE (cast(v["type"] as string) = 'WatchEvent') AND (year(cast(v["created_at"] as datetime)) >= 2015) --- GROUP BY --- repo, --- year --- ) t --- ) t2 --- WHERE r <= 10 --- ORDER BY --- year ASC, --- cnt DESC, repo --- \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.sql b/regression-test/suites/variant_github_events_new_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.sql deleted file mode 100644 index f0d95d00ead3fe..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.sql +++ /dev/null @@ -1,2 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ year(cast(v["created_at"] as datetime)) AS year, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY year ORDER BY year - diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments1.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments1.sql deleted file mode 100644 index 5c0e58e47ef961..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count() FROM github_events WHERE cast(v["type"] as string) = 'IssueCommentEvent' diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments2.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments2.sql deleted file mode 100644 index bb12557117b2c7..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() FROM github_events WHERE cast(v["type"] as string) = 'IssueCommentEvent' GROUP BY repo_name ORDER BY count() DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments3.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments3.sql deleted file mode 100644 index 1ab1e2969bfdb4..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments3.sql +++ /dev/null @@ -1,17 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - comments, - issues, - round(comments / issues, 2) AS ratio -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - count() AS comments, - count(distinct cast(v["payload"]["issue"]["number"] as int)) AS issues - FROM github_events - WHERE cast(v["type"] as string) = 'IssueCommentEvent' - GROUP BY repo_name -) t -ORDER BY comments DESC, 1, 3, 4 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments4.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments4.sql deleted file mode 100644 index 62bba924a9926c..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments4.sql +++ /dev/null @@ -1,9 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - cast(v["payload"]["issue"]["number"] as int) as number, - count() AS comments -FROM github_events -WHERE cast(v["type"] as string) = 'IssueCommentEvent' AND (cast(v["payload"]["action"] as string) = 'created') -GROUP BY repo_name, number -ORDER BY comments DESC, number ASC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments5.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments5.sql deleted file mode 100644 index fb8f3866328549..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments5.sql +++ /dev/null @@ -1,9 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - cast(v["payload"]["issue"]["number"] as int) as number, - count() AS comments -FROM github_events -WHERE cast(v["type"] as string) = 'IssueCommentEvent' AND (cast(v["payload"]["action"] as string) = 'created') AND (cast(v["payload"]["issue"]["number"] as int) > 10) -GROUP BY repo_name, number -ORDER BY comments DESC, repo_name, number -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments6.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments6.sql deleted file mode 100644 index 3782b46cd001c0..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments6.sql +++ /dev/null @@ -1,11 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - cast(v["payload"]["issue"]["number"] as int) as number, - count() AS comments, - count(distinct cast(v["actor"]["login"] as string)) AS authors -FROM github_events -WHERE cast(v["type"] as string) = 'IssueCommentEvent' AND (cast(v["payload"]["action"] as string) = 'created') AND (cast(v["payload"]["issue"]["number"] as int) > 10) -GROUP BY repo_name, number -HAVING authors >= 4 -ORDER BY comments DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments7.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments7.sql deleted file mode 100644 index b429dcc8bfe4f7..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments7.sql +++ /dev/null @@ -1,9 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS comments, - count(distinct cast(v["actor"]["login"] as string)) AS authors -FROM github_events -WHERE cast(v["type"] as string) = 'CommitCommentEvent' -GROUP BY repo_name -ORDER BY count() DESC, 1, 3 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments8.sql b/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments8.sql deleted file mode 100644 index 762a7dec440a63..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/issuesWithTheMostComments8.sql +++ /dev/null @@ -1,13 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- concat('https://github.com/', cast(v["repo"]["name"] as string), '/commit/', cast(v["payload"]["commit_id"] as string)) URL, --- cast(v["payload"]["commit_id"] as string) AS commit_id, --- count() AS comments, --- count(distinct cast(v["actor"]["login"] as string)) AS authors --- FROM github_events --- WHERE (cast(v["type"] as string) = 'CommitCommentEvent') AND commit_id != "" --- GROUP BY --- cast(v["repo"]["name"] as string), --- commit_id --- HAVING authors >= 10 --- ORDER BY count() DESC, URL, authors --- LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/mostForkedRepositories.sql b/regression-test/suites/variant_github_events_new_p2/sql/mostForkedRepositories.sql deleted file mode 100644 index f0ccb6e2c3d8fd..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/mostForkedRepositories.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS forks FROM github_events WHERE cast(v["type"] as string) = 'ForkEvent' GROUP BY repo_name ORDER BY forks DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/mostPopularCommentsOnGithub.sql b/regression-test/suites/variant_github_events_new_p2/sql/mostPopularCommentsOnGithub.sql deleted file mode 100644 index 8b7094ac153b9f..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/mostPopularCommentsOnGithub.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["payload"]["comment"]["body"] as string) as comment_body, count() FROM github_events WHERE cast(v["payload"]["comment"]["body"] as string) != "" AND length(cast(v["payload"]["comment"]["body"] as string)) < 100 GROUP BY comment_body ORDER BY count(), 1 DESC LIMIT 50 \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheNumberOfRepositories.sql b/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheNumberOfRepositories.sql deleted file mode 100644 index f6d653413c3dc5..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheNumberOfRepositories.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - lower(split_part(repo_name, '/', 1)) AS org, - count(distinct repo_name) AS repos -FROM -( - SELECT cast(v["repo"]["name"] as string) as repo_name - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' - GROUP BY repo_name - HAVING count() >= 10 -) t -GROUP BY org -ORDER BY repos DESC, org ASC -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheNumberOfStars.sql b/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheNumberOfStars.sql deleted file mode 100644 index 04b55d39e51291..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheNumberOfStars.sql +++ /dev/null @@ -1,8 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - lower(split_part(cast(v["repo"]["name"] as string), '/', 1)) AS org, - count() AS stars -FROM github_events -WHERE cast(v["type"] as string) = 'WatchEvent' -GROUP BY org -ORDER BY stars DESC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheSizeOfCommunity.sql b/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheSizeOfCommunity.sql deleted file mode 100644 index e0352afc55fa0f..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/organizationsByTheSizeOfCommunity.sql +++ /dev/null @@ -1,23 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- lower(split_part(cast(v["repo"]["name"] as string), '/', 1)) AS org, --- count(distinct cast(v["actor"]["login"] as string)) AS authors, --- count(distinct pr_author) AS pr_authors, --- count(distinct issue_author) AS issue_authors, --- count(distinct comment_author) AS comment_authors, --- count(distinct review_author) AS review_authors, --- count(distinct push_author) AS push_authors --- FROM --- ( --- SELECT --- cast(v["repo"]["name"] as string), --- cast(v["actor"]["login"] as string), --- CASE WHEN cast(v["type"] as string) = 'PullRequestEvent' THEN cast(v["actor"]["login"] as string) ELSE NULL END pr_author, --- CASE WHEN cast(v["type"] as string) = 'IssuesEvent' THEN cast(v["actor"]["login"] as string) ELSE NULL END issue_author, --- CASE WHEN cast(v["type"] as string) = 'IssueCommentEvent' THEN cast(v["actor"]["login"] as string) ELSE NULL END comment_author, --- CASE WHEN cast(v["type"] as string) = 'PullRequestReviewCommentEvent' THEN cast(v["actor"]["login"] as string) ELSE NULL END review_author, --- CASE WHEN cast(v["type"] as string) = 'PushEvent' THEN cast(v["actor"]["login"] as string) ELSE NULL END push_author --- FROM github_events --- WHERE cast(v["type"] as string) IN ('PullRequestEvent', 'IssuesEvent', 'IssueCommentEvent', 'PullRequestReviewCommentEvent', 'PushEvent') --- ) t --- GROUP BY org --- ORDER BY authors DESC diff --git a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks1.sql b/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks1.sql deleted file mode 100644 index 9e3f2875ef2e10..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks1.sql +++ /dev/null @@ -1,17 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(fork) AS forks, - sum(star) AS stars, - round(sum(star) / sum(fork), 3) AS ratio -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'ForkEvent' THEN 1 ELSE 0 END AS fork, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events - WHERE cast(v["type"] as string) IN ('ForkEvent', 'WatchEvent') -) t -GROUP BY repo_name -ORDER BY forks DESC, 1, 3, 4 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks2.sql b/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks2.sql deleted file mode 100644 index 2e1cb1b9c1f4e5..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks2.sql +++ /dev/null @@ -1,18 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(fork) AS forks, - sum(star) AS stars, - round(sum(star) / sum(fork), 3) AS ratio -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'ForkEvent' THEN 1 ELSE 0 END AS fork, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events - WHERE cast(v["type"] as string) IN ('ForkEvent', 'WatchEvent') -) t -GROUP BY repo_name -HAVING (stars > 20) AND (forks >= 10) -ORDER BY ratio DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks3.sql b/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks3.sql deleted file mode 100644 index 6f91ab15eb9823..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks3.sql +++ /dev/null @@ -1,18 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(fork) AS forks, - sum(star) AS stars, - round(sum(fork) / sum(star), 2) AS ratio -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'ForkEvent' THEN 1 ELSE 0 END AS fork, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events - WHERE cast(v["type"] as string) IN ('ForkEvent', 'WatchEvent') -) t -GROUP BY repo_name -HAVING (stars > 4) AND (forks > 4) -ORDER BY ratio DESC -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks4.sql b/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks4.sql deleted file mode 100644 index 8be41f0949737f..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks4.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - sum(fork) AS forks, - sum(star) AS stars, - round(sum(star) / sum(fork), 2) AS ratio -FROM -( - SELECT - cast(v["repo"]["name"] as string), - CASE WHEN cast(v["type"] as string) = 'ForkEvent' THEN 1 ELSE 0 END AS fork, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events - WHERE cast(v["type"] as string) IN ('ForkEvent', 'WatchEvent') -) t diff --git a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks5.sql b/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks5.sql deleted file mode 100644 index 26bfb2ac6302ee..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/proportionsBetweenStarsAndForks5.sql +++ /dev/null @@ -1,21 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - sum(forks) AS forks, - sum(stars) AS stars, - round(sum(stars) / sum(forks), 2) AS ratio -FROM -( - SELECT - sum(fork) AS forks, - sum(star) AS stars - FROM - ( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'ForkEvent' THEN 1 ELSE 0 END AS fork, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events - WHERE cast(v["type"] as string) IN ('ForkEvent', 'WatchEvent') - ) t - GROUP BY repo_name - HAVING stars > 10 -) t2 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesByTheNumberOfPushes.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesByTheNumberOfPushes.sql deleted file mode 100644 index 6db36e257c5d45..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesByTheNumberOfPushes.sql +++ /dev/null @@ -1,17 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS pushes, - count(distinct cast(v["actor"]["login"] as string)) AS authors -FROM github_events -WHERE (cast(v["type"] as string) = 'PushEvent') AND (cast(v["repo"]["name"] as string) IN -( - SELECT cast(v["repo"]["name"] as string) as repo_name - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' - GROUP BY repo_name - ORDER BY count() DESC - LIMIT 10000 -)) -GROUP BY repo_name -ORDER BY count() DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithClickhouse_related_comments1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithClickhouse_related_comments1.sql deleted file mode 100644 index 254888d0e11901..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithClickhouse_related_comments1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() FROM github_events WHERE lower(cast(v["payload"]["comment"]["body"] as string)) LIKE '%apache%' GROUP BY repo_name ORDER BY count() DESC, repo_name ASC LIMIT 50 \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithClickhouse_related_comments2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithClickhouse_related_comments2.sql deleted file mode 100644 index 90ef187ba3520c..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithClickhouse_related_comments2.sql +++ /dev/null @@ -1,17 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(num_star) AS num_stars, - sum(num_comment) AS num_comments -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS num_star, - CASE WHEN lower(cast(v["payload"]["comment"]["body"] as string)) LIKE '%apache%' THEN 1 ELSE 0 END AS num_comment - FROM github_events - WHERE (lower(cast(v["payload"]["comment"]["body"] as string)) LIKE '%apache%') OR (cast(v["type"] as string) = 'WatchEvent') -) t -GROUP BY repo_name -HAVING num_comments > 0 -ORDER BY num_stars DESC,num_comments DESC, repo_name ASC -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithDoris_related_comments1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithDoris_related_comments1.sql deleted file mode 100644 index 90a73db9ca3897..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithDoris_related_comments1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() FROM github_events WHERE lower(cast(v["payload"]["comment"]["body"] as string)) LIKE '%spark%' GROUP BY repo_name ORDER BY count() DESC, repo_name ASC LIMIT 50 \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithDoris_related_comments2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithDoris_related_comments2.sql deleted file mode 100644 index 28c44705455d23..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithDoris_related_comments2.sql +++ /dev/null @@ -1,17 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(num_star) AS num_stars, - sum(num_comment) AS num_comments -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS num_star, - CASE WHEN lower(cast(v["payload"]["comment"]["body"] as string)) LIKE '%spark%' THEN 1 ELSE 0 END AS num_comment - FROM github_events - WHERE (lower(cast(v["payload"]["comment"]["body"] as string)) LIKE '%spark%') OR (cast(v["type"] as string) = 'WatchEvent') -) t -GROUP BY repo_name -HAVING num_comments > 0 -ORDER BY num_stars DESC,num_comments DESC,repo_name ASC -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheHighestGrowthYoY.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheHighestGrowthYoY.sql deleted file mode 100644 index 47d3ad8417dc4f..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheHighestGrowthYoY.sql +++ /dev/null @@ -1,20 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(created_at_2022) AS stars2022, - sum(created_at_2015) AS stars2015, - round(sum(created_at_2022) / sum(created_at_2015), 3) AS yoy, - min(created_at) AS first_seen -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE year(cast(v["created_at"] as datetime)) WHEN 2022 THEN 1 ELSE 0 END AS created_at_2022, - CASE year(cast(v["created_at"] as datetime)) WHEN 2015 THEN 1 ELSE 0 END AS created_at_2015, - cast(v["created_at"] as datetime) as created_at - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' -) t -GROUP BY repo_name -HAVING (min(created_at) <= '2023-01-01 00:00:00') AND (stars2022 >= 1) and (stars2015 >= 1) -ORDER BY yoy DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.sql deleted file mode 100644 index d93b20c783fe11..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS c, count(distinct cast(v["actor"]["login"] as string)) AS u FROM github_events WHERE cast(v["type"] as string) = 'IssuesEvent' AND cast(v["payload"]["action"] as string) = 'opened' GROUP BY repo_name ORDER BY c DESC, repo_name LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.sql deleted file mode 100644 index 5703e50af8f5d9..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.sql +++ /dev/null @@ -1,18 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(issue_created) AS c, - count(distinct actor_login) AS u, - sum(star) AS stars -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') THEN 1 ELSE 0 END AS issue_created, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star, - CASE WHEN (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') THEN cast(v["actor"]["login"] as string) ELSE NULL END AS actor_login - FROM github_events - WHERE cast(v["type"] as string) IN ('IssuesEvent', 'WatchEvent') -) t -GROUP BY repo_name -ORDER BY c DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.sql deleted file mode 100644 index 32014cb4c195bc..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.sql +++ /dev/null @@ -1,19 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(issue_created) AS c, - count(distinct actor_login) AS u, - sum(star) AS stars -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') THEN 1 ELSE 0 END AS issue_created, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star, - CASE WHEN (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') THEN cast(v["actor"]["login"] as string) ELSE NULL END AS actor_login - FROM github_events - WHERE cast(v["type"] as string) IN ('IssuesEvent', 'WatchEvent') -) t -GROUP BY repo_name -HAVING stars >= 10 -ORDER BY c, u, stars DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.sql deleted file mode 100644 index 484d8b50ea8ccd..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.sql +++ /dev/null @@ -1,18 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(issue_created) AS c, - count(distinct actor_login) AS u, - sum(star) AS stars -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') THEN 1 ELSE 0 END AS issue_created, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star, - CASE WHEN (cast(v["type"] as string) = 'IssuesEvent') AND (cast(v["payload"]["action"] as string) = 'opened') THEN cast(v["actor"]["login"] as string) ELSE NULL END AS actor_login - FROM github_events - WHERE cast(v["type"] as string) IN ('IssuesEvent', 'WatchEvent') -) t -GROUP BY repo_name -ORDER BY u, c, stars DESC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.sql deleted file mode 100644 index 688d42ec6a94b4..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count(), count(distinct cast(v["actor"]["login"] as string)) FROM github_events WHERE cast(v["type"] as string) = 'PullRequestEvent' AND cast(v["payload"]["action"] as string) = 'opened' GROUP BY repo_name ORDER BY 2,1,3 DESC LIMIT 50 \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.sql deleted file mode 100644 index e4b3b388e24159..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count(), count(distinct cast(v["actor"]["login"] as string)) AS u FROM github_events WHERE cast(v["type"] as string) = 'PullRequestEvent' AND cast(v["payload"]["action"] as string) = 'opened' GROUP BY repo_name ORDER BY u DESC, 2 DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql deleted file mode 100644 index f6672875578f47..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql +++ /dev/null @@ -1,17 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(invitation) AS invitations, - sum(star) AS stars -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE WHEN cast(v["type"] as string) = 'MemberEvent' THEN 1 ELSE 0 END AS invitation, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events - WHERE cast(v["type"] as string) IN ('MemberEvent', 'WatchEvent') -) t -GROUP BY repo_name -HAVING stars >= 2 -ORDER BY invitations DESC, stars DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql deleted file mode 100644 index d31a0f38a54a13..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - count(distinct actor_login) AS u, - sum(star) AS stars -FROM -( - SELECT - lower(cast(v["repo"]["name"] as string)) as repo_name, - CASE WHEN cast(v["type"] as string) = 'PushEvent' THEN cast(v["actor"]["login"] as string) ELSE NULL END AS actor_login, - CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star - FROM github_events WHERE cast(v["type"] as string) IN ('PushEvent', 'WatchEvent') AND cast(v["repo"]["name"] as string) != '/' -) t -GROUP BY repo_name ORDER BY u, stars, repo_name DESC LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql deleted file mode 100644 index 5849b3527d3a55..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql +++ /dev/null @@ -1,13 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- cast(v["repo"]["name"] as string), --- count(distinct cast(v["actor"]["login"] as string)) AS u, --- sum(star) AS stars --- FROM --- ( --- SELECT --- cast(v["repo"]["name"] as string), --- CASE WHEN cast(v["type"] as string) = 'PushEvent' AND (ref LIKE '%/master' OR ref LIKE '%/main') THEN cast(v["actor"]["login"] as string) ELSE NULL END AS cast(v["actor"]["login"] as string), --- CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star --- FROM github_events WHERE cast(v["type"] as string) IN ('PushEvent', 'WatchEvent') AND cast(v["repo"]["name"] as string) != '/' --- ) t --- GROUP BY cast(v["repo"]["name"] as string) ORDER BY u DESC LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql deleted file mode 100644 index b3b2133eb2991c..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql +++ /dev/null @@ -1,16 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- cast(v["repo"]["name"] as string), --- count(distinct cast(v["actor"]["login"] as string)) AS u, --- sum(star) AS stars --- FROM --- ( --- SELECT --- cast(v["repo"]["name"] as string), --- CASE WHEN cast(v["type"] as string) = 'PushEvent' AND (ref LIKE '%/master' OR ref LIKE '%/main') THEN cast(v["actor"]["login"] as string) ELSE NULL END AS cast(v["actor"]["login"] as string), --- CASE WHEN cast(v["type"] as string) = 'WatchEvent' THEN 1 ELSE 0 END AS star --- FROM github_events WHERE cast(v["type"] as string) IN ('PushEvent', 'WatchEvent') AND cast(v["repo"]["name"] as string) != '/' --- ) t --- GROUP BY cast(v["repo"]["name"] as string) --- HAVING stars >= 100 --- ORDER BY u DESC --- LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay1.sql deleted file mode 100644 index dcf6eb23235a69..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay1.sql +++ /dev/null @@ -1,25 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - day, - stars -FROM -( - SELECT - row_number() OVER (PARTITION BY repo_name ORDER BY stars DESC) AS rank, - repo_name, - day, - stars - FROM - ( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - to_date(cast(v["created_at"] as datetime)) AS day, - count() AS stars - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' - GROUP BY repo_name, day - ) t1 -) t2 -WHERE rank = 1 -ORDER BY stars DESC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay2.sql deleted file mode 100644 index 66a7ab7cba55fd..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay2.sql +++ /dev/null @@ -1,25 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- cast(v["repo"]["name"] as string), --- day, --- stars --- FROM --- ( --- SELECT --- row_number() OVER (PARTITION BY cast(v["repo"]["name"] as string) ORDER BY stars DESC) AS rank, --- cast(v["repo"]["name"] as string), --- day, --- stars --- FROM --- ( --- SELECT --- cast(v["repo"]["name"] as string), --- to_date(cast(v["created_at"] as datetime)) AS day, --- count() AS stars --- FROM github_events --- WHERE cast(v["type"] as string) = 'WatchEvent' --- GROUP BY cast(v["repo"]["name"] as string), day --- ) t1 --- ) t2 --- WHERE rank = 1 --- ORDER BY stars DESC --- LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay3.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay3.sql deleted file mode 100644 index 1bfa7869399474..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostStarsOverOneDay3.sql +++ /dev/null @@ -1 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string), cast(v["created_at"] as datetime), count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY cast(v["repo"]["name"] as string), cast(v["created_at"] as datetime) ORDER BY count() DESC LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.sql deleted file mode 100644 index d1abdfd9dd4e0e..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.sql +++ /dev/null @@ -1,20 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - max(stars) AS daily_stars, - sum(stars) AS total_stars, - sum(stars) / max(stars) AS rate -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - to_date(cast(v["created_at"] as datetime)) AS day, - count() AS stars - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' - GROUP BY - repo_name, - day -) t -GROUP BY repo_name -ORDER BY rate DESC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheWorstStagnation_order.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheWorstStagnation_order.sql deleted file mode 100644 index a4c09020695e9d..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesWithTheWorstStagnation_order.sql +++ /dev/null @@ -1,20 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - sum(created_at_2022) AS stars2022, - sum(created_at_2015) AS stars2015, - round(sum(created_at_2022) / sum(created_at_2015), 3) AS yoy, - min(created_at) AS first_seen -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - CASE year(cast(v["created_at"] as datetime)) WHEN 2022 THEN 1 ELSE 0 END AS created_at_2022, - CASE year(cast(v["created_at"] as datetime)) WHEN 2015 THEN 1 ELSE 0 END AS created_at_2015, - cast(v["created_at"] as datetime) as created_at - FROM github_events - WHERE cast(v["type"] as string) = 'WatchEvent' -) t -GROUP BY repo_name -HAVING (min(created_at) <= '2019-01-01 00:00:00') AND (max(created_at) >= '2020-06-01 00:00:00') AND (stars2015 >= 2) -ORDER BY yoy, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoryAffinityList1.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoryAffinityList1.sql deleted file mode 100644 index f75b3d0d493adb..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoryAffinityList1.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS stars -FROM github_events -WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["actor"]["login"] as string) IN -( - SELECT cast(v["actor"]["login"] as string) - FROM github_events - WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["repo"]["name"] as string) IN ('apache/spark', 'prakhar1989/awesome-courses')) -)) AND (cast(v["repo"]["name"] as string) NOT IN ('ClickHouse/ClickHouse', 'yandex/ClickHouse')) -GROUP BY repo_name -ORDER BY stars DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoryAffinityList2.sql b/regression-test/suites/variant_github_events_new_p2/sql/repositoryAffinityList2.sql deleted file mode 100644 index 6f5ae4cbb33dc7..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/repositoryAffinityList2.sql +++ /dev/null @@ -1,23 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - repo_name, - total_stars, - round(spark_stars / total_stars, 2) AS ratio -FROM -( - SELECT - cast(v["repo"]["name"] as string) as repo_name, - count(distinct cast(v["actor"]["login"] as string)) AS total_stars - FROM github_events - WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["repo"]["name"] as string) NOT IN ('apache/spark')) - GROUP BY repo_name - HAVING total_stars >= 10 -) t1 -JOIN -( - SELECT - count(distinct cast(v["actor"]["login"] as string)) AS spark_stars - FROM github_events - WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["repo"]["name"] as string) IN ('apache/spark')) -) t2 -ORDER BY ratio DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/starsFromHeavyGithubUsers1.sql b/regression-test/suites/variant_github_events_new_p2/sql/starsFromHeavyGithubUsers1.sql deleted file mode 100644 index 0cd0b5ade503a3..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/starsFromHeavyGithubUsers1.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() -FROM github_events -WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["actor"]["login"] as string) IN -( - SELECT cast(v["actor"]["login"] as string) - FROM github_events - WHERE (cast(v["type"] as string) = 'PullRequestEvent') AND (cast(v["payload"]["action"] as string) = 'opened') -)) -GROUP BY repo_name -ORDER BY count() DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/starsFromHeavyGithubUsers2.sql b/regression-test/suites/variant_github_events_new_p2/sql/starsFromHeavyGithubUsers2.sql deleted file mode 100644 index d230a3186fbe93..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/starsFromHeavyGithubUsers2.sql +++ /dev/null @@ -1,15 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() -FROM github_events -WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["actor"]["login"] as string) IN -( - SELECT cast(v["actor"]["login"] as string) as actor_login - FROM github_events - WHERE (cast(v["type"] as string) = 'PullRequestEvent') AND (cast(v["payload"]["action"] as string) = 'opened') - GROUP BY actor_login - HAVING count() >= 2 -)) -GROUP BY repo_name -ORDER BY 1, count() DESC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/test_distinct_streaming_agg.sql b/regression-test/suites/variant_github_events_new_p2/sql/test_distinct_streaming_agg.sql deleted file mode 100644 index 40854a2d53502f..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/test_distinct_streaming_agg.sql +++ /dev/null @@ -1,5 +0,0 @@ -SELECT - /*+SET_VAR(batch_size=50,experimental_enable_pipeline_x_engine=false,parallel_pipeline_task_num=1,disable_streaming_preaggregations=false) */ - count(distinct v) -FROM - github_events_2; \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theLongestRepositoryNames1.sql b/regression-test/suites/variant_github_events_new_p2/sql/theLongestRepositoryNames1.sql deleted file mode 100644 index 4644f27b54f784..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theLongestRepositoryNames1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count(), cast(v["repo"]["name"] as string) as repo_name FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY repo_name ORDER BY length(repo_name) DESC, repo_name LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theLongestRepositoryNames2.sql b/regression-test/suites/variant_github_events_new_p2/sql/theLongestRepositoryNames2.sql deleted file mode 100644 index ee34ddfaae05a1..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theLongestRepositoryNames2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND cast(v["repo"]["name"] as string) LIKE '%_/_%' GROUP BY repo_name ORDER BY length(repo_name) ASC, repo_name LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theMostToughCodeReviews.sql b/regression-test/suites/variant_github_events_new_p2/sql/theMostToughCodeReviews.sql deleted file mode 100644 index 29cf3d807c3760..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theMostToughCodeReviews.sql +++ /dev/null @@ -1,10 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - concat('https://github.com/', cast(v["repo"]["name"] as string), '/pull/') AS URL, - count(distinct cast(v["actor"]["login"] as string)) AS authors -FROM github_events -WHERE (cast(v["type"] as string) = 'PullRequestReviewCommentEvent') AND (cast(v["payload"]["action"] as string) = 'created') -GROUP BY - cast(v["repo"]["name"] as string), - cast(v["payload"]["issue"]["number"] as string) -ORDER BY authors DESC, URL ASC -LIMIT 50 \ No newline at end of file diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfRepositoriesOnGithub.sql b/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfRepositoriesOnGithub.sql deleted file mode 100644 index e10d25749b48bc..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfRepositoriesOnGithub.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count(distinct cast(v["repo"]["name"] as string)) FROM github_events diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub1.sql b/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub1.sql deleted file mode 100644 index fe5f1956c7948a..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count(distinct cast(v["actor"]["login"] as string)) FROM github_events diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub2.sql b/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub2.sql deleted file mode 100644 index 72dccfb12d697e..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count(distinct cast(v["actor"]["login"] as string)) FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub3.sql b/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub3.sql deleted file mode 100644 index ce96a89bd27105..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub3.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count(distinct cast(v["actor"]["login"] as string)) FROM github_events WHERE cast(v["type"] as string) = 'PushEvent' diff --git a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub4.sql b/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub4.sql deleted file mode 100644 index 53e611219f3dd0..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/theTotalNumberOfUsersOnGithub4.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ count(distinct cast(v["actor"]["login"] as string)) FROM github_events WHERE cast(v["type"] as string) = 'PullRequestEvent' AND cast(v["payload"]["action"] as string) = 'opened' diff --git a/regression-test/suites/variant_github_events_new_p2/sql/topLabels1.sql b/regression-test/suites/variant_github_events_new_p2/sql/topLabels1.sql deleted file mode 100644 index eb6dd56d26c556..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/topLabels1.sql +++ /dev/null @@ -1,9 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- label, --- count() AS c --- FROM github_events --- LATERAL VIEW explode_split(labels, ',') t AS label --- WHERE (cast(v["type"] as string) IN ('IssuesEvent', 'PullRequestEvent', 'IssueCommentEvent')) AND (action IN ('created', 'opened', 'labeled')) --- GROUP BY label --- ORDER BY c DESC --- LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/topLabels2.sql b/regression-test/suites/variant_github_events_new_p2/sql/topLabels2.sql deleted file mode 100644 index d04bded901d30c..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/topLabels2.sql +++ /dev/null @@ -1,9 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- label, --- count() AS c --- FROM github_events --- LATERAL VIEW explode_split(labels, ',') t AS label --- WHERE (cast(v["type"] as string) IN ('IssuesEvent', 'PullRequestEvent', 'IssueCommentEvent')) AND (action IN ('created', 'opened', 'labeled')) AND ((lower(label) LIKE '%bug%') OR (lower(label) LIKE '%feature%')) --- GROUP BY label --- ORDER BY c DESC --- LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/topLabels3.sql b/regression-test/suites/variant_github_events_new_p2/sql/topLabels3.sql deleted file mode 100644 index f35116aaa6d27f..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/topLabels3.sql +++ /dev/null @@ -1,14 +0,0 @@ --- SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ --- sum(bug) AS bugs, --- sum(feature) AS feature, --- sum(bug) / sum(feature) AS ratio --- FROM --- ( --- SELECT --- CASE WHEN lower(label) LIKE '%bug%' THEN 1 ELSE 0 END AS bug, --- CASE WHEN lower(label) LIKE '%feature%' THEN 1 ELSE 0 END AS feature --- FROM github_events --- LATERAL VIEW explode_split(labels, ',') t AS label --- WHERE (cast(v["type"] as string) IN ('IssuesEvent', 'PullRequestEvent', 'IssueCommentEvent')) AND (action IN ('created', 'opened', 'labeled')) AND ((lower(label) LIKE '%bug%') OR (lower(label) LIKE '%feature%')) --- ) t --- LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/topRepositoriesByStars.sql b/regression-test/suites/variant_github_events_new_p2/sql/topRepositoriesByStars.sql deleted file mode 100644 index 41eec01ee97809..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/topRepositoriesByStars.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["repo"]["name"] as string) as repo_name, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY repo_name ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.sql b/regression-test/suites/variant_github_events_new_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.sql deleted file mode 100644 index c0d4576ce60088..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ dayofweek(cast(v["created_at"] as datetime)) AS day, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY day ORDER BY day diff --git a/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars1.sql b/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars1.sql deleted file mode 100644 index 91f186645603d6..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars1.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["actor"]["login"] as string) as actor_login, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' GROUP BY actor_login ORDER BY stars DESC, 1 LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars2.sql b/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars2.sql deleted file mode 100644 index 30a9ea294ccf9e..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars2.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ cast(v["actor"]["login"] as string) as actor_login, count() AS stars FROM github_events WHERE cast(v["type"] as string) = 'WatchEvent' AND cast(v["actor"]["login"] as string) = 'cliffordfajardo' GROUP BY actor_login ORDER BY stars DESC LIMIT 50 diff --git a/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars3.sql b/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars3.sql deleted file mode 100644 index 524737737b6bac..00000000000000 --- a/regression-test/suites/variant_github_events_new_p2/sql/whoAreAllThosePeopleGivingStars3.sql +++ /dev/null @@ -1,13 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS stars -FROM github_events -WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["repo"]["name"] as string) IN -( - SELECT cast(v["repo"]["name"] as string) - FROM github_events - WHERE (cast(v["type"] as string) = 'WatchEvent') AND (cast(v["actor"]["login"] as string) = 'cliffordfajardo') -)) -GROUP BY repo_name -ORDER BY stars DESC, repo_name -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_p2/repositoriesByAmountOfModifiedCode.sql b/regression-test/suites/variant_github_events_p2/repositoriesByAmountOfModifiedCode.sql deleted file mode 100644 index 6c1632769f3d9f..00000000000000 --- a/regression-test/suites/variant_github_events_p2/repositoriesByAmountOfModifiedCode.sql +++ /dev/null @@ -1,12 +0,0 @@ -SELECT /*+SET_VAR(enable_fallback_to_original_planner=false,disable_streaming_preaggregations=false) */ - cast(v["repo"]["name"] as string) as repo_name, - count() AS prs, - count(distinct cast(v["actor"]["login"] as string)) AS authors, - sum(cast(v["payload"]["pull_request"]["additions"] as int)) AS adds, - sum(cast(v["payload"]["pull_request"]["deletions"] as int)) AS dels -FROM github_events -WHERE (cast(v["type"] as string) = 'PullRequestEvent') AND (cast(v["payload"]["action"] as string) = 'opened') AND (cast(v["payload"]["pull_request"]["additions"] as int) < 10000) AND (cast(v["payload"]["pull_request"]["deletions"] as int) < 10000) -GROUP BY repo_name -HAVING (adds / dels) < 10 -ORDER BY adds + dels DESC, 1 -LIMIT 50 diff --git a/regression-test/suites/variant_github_events_p2/affinityByIssuesAndPRs1.sql b/regression-test/suites/variant_github_events_p2/sql/affinityByIssuesAndPRs1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/affinityByIssuesAndPRs1.sql rename to regression-test/suites/variant_github_events_p2/sql/affinityByIssuesAndPRs1.sql diff --git a/regression-test/suites/variant_github_events_p2/affinityByIssuesAndPRs2.sql b/regression-test/suites/variant_github_events_p2/sql/affinityByIssuesAndPRs2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/affinityByIssuesAndPRs2.sql rename to regression-test/suites/variant_github_events_p2/sql/affinityByIssuesAndPRs2.sql diff --git a/regression-test/suites/variant_github_events_p2/authorsWithTheMostPushes.sql b/regression-test/suites/variant_github_events_p2/sql/authorsWithTheMostPushes.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/authorsWithTheMostPushes.sql rename to regression-test/suites/variant_github_events_p2/sql/authorsWithTheMostPushes.sql diff --git a/regression-test/suites/variant_github_events_p2/countingStar1.sql b/regression-test/suites/variant_github_events_p2/sql/countingStar1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/countingStar1.sql rename to regression-test/suites/variant_github_events_p2/sql/countingStar1.sql diff --git a/regression-test/suites/variant_github_events_p2/countingStar2.sql b/regression-test/suites/variant_github_events_p2/sql/countingStar2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/countingStar2.sql rename to regression-test/suites/variant_github_events_p2/sql/countingStar2.sql diff --git a/regression-test/suites/variant_github_events_p2/countingStar3.sql b/regression-test/suites/variant_github_events_p2/sql/countingStar3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/countingStar3.sql rename to regression-test/suites/variant_github_events_p2/sql/countingStar3.sql diff --git a/regression-test/suites/variant_github_events_p2/distributionOfRepositoriesByStarCount.sql b/regression-test/suites/variant_github_events_p2/sql/distributionOfRepositoriesByStarCount.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/distributionOfRepositoriesByStarCount.sql rename to regression-test/suites/variant_github_events_p2/sql/distributionOfRepositoriesByStarCount.sql diff --git a/regression-test/suites/variant_github_events_p2/githubRoulette.sql b/regression-test/suites/variant_github_events_p2/sql/githubRoulette.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/githubRoulette.sql rename to regression-test/suites/variant_github_events_p2/sql/githubRoulette.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears1.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears2.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears3.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears4.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears5.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears6.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheListOfTopRepositoriesChangedOverTheYears7.sql diff --git a/regression-test/suites/variant_github_events_p2/howHasTheTotalNumberOfStarsChangedOverTime.sql b/regression-test/suites/variant_github_events_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/howHasTheTotalNumberOfStarsChangedOverTime.sql rename to regression-test/suites/variant_github_events_p2/sql/howHasTheTotalNumberOfStarsChangedOverTime.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments1.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments1.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments1.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments2.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments2.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments2.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments3.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments3.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments3.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments4.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments4.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments4.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments4.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments5.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments5.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments5.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments5.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments6.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments6.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments6.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments6.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments7.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments7.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments7.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments7.sql diff --git a/regression-test/suites/variant_github_events_p2/issuesWithTheMostComments8.sql b/regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments8.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/issuesWithTheMostComments8.sql rename to regression-test/suites/variant_github_events_p2/sql/issuesWithTheMostComments8.sql diff --git a/regression-test/suites/variant_github_events_p2/mostForkedRepositories.sql b/regression-test/suites/variant_github_events_p2/sql/mostForkedRepositories.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/mostForkedRepositories.sql rename to regression-test/suites/variant_github_events_p2/sql/mostForkedRepositories.sql diff --git a/regression-test/suites/variant_github_events_p2/mostPopularCommentsOnGithub.sql b/regression-test/suites/variant_github_events_p2/sql/mostPopularCommentsOnGithub.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/mostPopularCommentsOnGithub.sql rename to regression-test/suites/variant_github_events_p2/sql/mostPopularCommentsOnGithub.sql diff --git a/regression-test/suites/variant_github_events_p2/organizationsByTheNumberOfRepositories.sql b/regression-test/suites/variant_github_events_p2/sql/organizationsByTheNumberOfRepositories.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/organizationsByTheNumberOfRepositories.sql rename to regression-test/suites/variant_github_events_p2/sql/organizationsByTheNumberOfRepositories.sql diff --git a/regression-test/suites/variant_github_events_p2/organizationsByTheNumberOfStars.sql b/regression-test/suites/variant_github_events_p2/sql/organizationsByTheNumberOfStars.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/organizationsByTheNumberOfStars.sql rename to regression-test/suites/variant_github_events_p2/sql/organizationsByTheNumberOfStars.sql diff --git a/regression-test/suites/variant_github_events_p2/organizationsByTheSizeOfCommunity.sql b/regression-test/suites/variant_github_events_p2/sql/organizationsByTheSizeOfCommunity.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/organizationsByTheSizeOfCommunity.sql rename to regression-test/suites/variant_github_events_p2/sql/organizationsByTheSizeOfCommunity.sql diff --git a/regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks1.sql b/regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks1.sql rename to regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks1.sql diff --git a/regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks2.sql b/regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks2.sql rename to regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks2.sql diff --git a/regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks3.sql b/regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks3.sql rename to regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks3.sql diff --git a/regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks4.sql b/regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks4.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks4.sql rename to regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks4.sql diff --git a/regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks5.sql b/regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks5.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/proportionsBetweenStarsAndForks5.sql rename to regression-test/suites/variant_github_events_p2/sql/proportionsBetweenStarsAndForks5.sql diff --git a/regression-test/suites/variant_github_events_new_p2/sql/repositoriesByAmountOfModifiedCode.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesByAmountOfModifiedCode.sql similarity index 100% rename from regression-test/suites/variant_github_events_new_p2/sql/repositoriesByAmountOfModifiedCode.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesByAmountOfModifiedCode.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesByTheNumberOfPushes.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesByTheNumberOfPushes.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesByTheNumberOfPushes.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesByTheNumberOfPushes.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithClickhouse_related_comments1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithClickhouse_related_comments1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithClickhouse_related_comments2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithClickhouse_related_comments2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithClickhouse_related_comments2.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithDoris_related_comments1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithDoris_related_comments1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithDoris_related_comments1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithDoris_related_comments1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithDoris_related_comments2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithDoris_related_comments2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithDoris_related_comments2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithDoris_related_comments2.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheHighestGrowthYoY.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheHighestGrowthYoY.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheHighestGrowthYoY.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheHighestGrowthYoY.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues2.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues3.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues3.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues3.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues4.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfIssues4.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfIssues4.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumAmountOfPullRequests2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumAmountOfPullRequests2.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMaximumNumberOfAcceptedInvitations.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess2.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostPeopleWhoHavePushAccess3.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay2.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay3.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostStarsOverOneDay3.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostStarsOverOneDay3.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheMostSteadyGrowthOverTime.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheMostSteadyGrowthOverTime.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheMostSteadyGrowthOverTime.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoriesWithTheWorstStagnation_order.sql b/regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheWorstStagnation_order.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoriesWithTheWorstStagnation_order.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoriesWithTheWorstStagnation_order.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoryAffinityList1.sql b/regression-test/suites/variant_github_events_p2/sql/repositoryAffinityList1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoryAffinityList1.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoryAffinityList1.sql diff --git a/regression-test/suites/variant_github_events_p2/repositoryAffinityList2.sql b/regression-test/suites/variant_github_events_p2/sql/repositoryAffinityList2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/repositoryAffinityList2.sql rename to regression-test/suites/variant_github_events_p2/sql/repositoryAffinityList2.sql diff --git a/regression-test/suites/variant_github_events_p2/starsFromHeavyGithubUsers1.sql b/regression-test/suites/variant_github_events_p2/sql/starsFromHeavyGithubUsers1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/starsFromHeavyGithubUsers1.sql rename to regression-test/suites/variant_github_events_p2/sql/starsFromHeavyGithubUsers1.sql diff --git a/regression-test/suites/variant_github_events_p2/starsFromHeavyGithubUsers2.sql b/regression-test/suites/variant_github_events_p2/sql/starsFromHeavyGithubUsers2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/starsFromHeavyGithubUsers2.sql rename to regression-test/suites/variant_github_events_p2/sql/starsFromHeavyGithubUsers2.sql diff --git a/regression-test/suites/variant_github_events_p2/theLongestRepositoryNames1.sql b/regression-test/suites/variant_github_events_p2/sql/theLongestRepositoryNames1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theLongestRepositoryNames1.sql rename to regression-test/suites/variant_github_events_p2/sql/theLongestRepositoryNames1.sql diff --git a/regression-test/suites/variant_github_events_p2/theLongestRepositoryNames2.sql b/regression-test/suites/variant_github_events_p2/sql/theLongestRepositoryNames2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theLongestRepositoryNames2.sql rename to regression-test/suites/variant_github_events_p2/sql/theLongestRepositoryNames2.sql diff --git a/regression-test/suites/variant_github_events_p2/theMostToughCodeReviews.sql b/regression-test/suites/variant_github_events_p2/sql/theMostToughCodeReviews.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theMostToughCodeReviews.sql rename to regression-test/suites/variant_github_events_p2/sql/theMostToughCodeReviews.sql diff --git a/regression-test/suites/variant_github_events_p2/theTotalNumberOfRepositoriesOnGithub.sql b/regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfRepositoriesOnGithub.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theTotalNumberOfRepositoriesOnGithub.sql rename to regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfRepositoriesOnGithub.sql diff --git a/regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub1.sql b/regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub1.sql rename to regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub1.sql diff --git a/regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub2.sql b/regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub2.sql rename to regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub2.sql diff --git a/regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub3.sql b/regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub3.sql rename to regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub3.sql diff --git a/regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub4.sql b/regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub4.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/theTotalNumberOfUsersOnGithub4.sql rename to regression-test/suites/variant_github_events_p2/sql/theTotalNumberOfUsersOnGithub4.sql diff --git a/regression-test/suites/variant_github_events_p2/topLabels1.sql b/regression-test/suites/variant_github_events_p2/sql/topLabels1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/topLabels1.sql rename to regression-test/suites/variant_github_events_p2/sql/topLabels1.sql diff --git a/regression-test/suites/variant_github_events_p2/topLabels2.sql b/regression-test/suites/variant_github_events_p2/sql/topLabels2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/topLabels2.sql rename to regression-test/suites/variant_github_events_p2/sql/topLabels2.sql diff --git a/regression-test/suites/variant_github_events_p2/topLabels3.sql b/regression-test/suites/variant_github_events_p2/sql/topLabels3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/topLabels3.sql rename to regression-test/suites/variant_github_events_p2/sql/topLabels3.sql diff --git a/regression-test/suites/variant_github_events_p2/topRepositoriesByStars.sql b/regression-test/suites/variant_github_events_p2/sql/topRepositoriesByStars.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/topRepositoriesByStars.sql rename to regression-test/suites/variant_github_events_p2/sql/topRepositoriesByStars.sql diff --git a/regression-test/suites/variant_github_events_p2/whatIsTheBestDayOfTheWeekToCatchAStar.sql b/regression-test/suites/variant_github_events_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/whatIsTheBestDayOfTheWeekToCatchAStar.sql rename to regression-test/suites/variant_github_events_p2/sql/whatIsTheBestDayOfTheWeekToCatchAStar.sql diff --git a/regression-test/suites/variant_github_events_p2/whoAreAllThosePeopleGivingStars1.sql b/regression-test/suites/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars1.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/whoAreAllThosePeopleGivingStars1.sql rename to regression-test/suites/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars1.sql diff --git a/regression-test/suites/variant_github_events_p2/whoAreAllThosePeopleGivingStars2.sql b/regression-test/suites/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars2.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/whoAreAllThosePeopleGivingStars2.sql rename to regression-test/suites/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars2.sql diff --git a/regression-test/suites/variant_github_events_p2/whoAreAllThosePeopleGivingStars3.sql b/regression-test/suites/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars3.sql similarity index 100% rename from regression-test/suites/variant_github_events_p2/whoAreAllThosePeopleGivingStars3.sql rename to regression-test/suites/variant_github_events_p2/sql/whoAreAllThosePeopleGivingStars3.sql diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh index 1bebdac7769f85..8f6a9fafc71757 100755 --- a/thirdparty/download-thirdparty.sh +++ b/thirdparty/download-thirdparty.sh @@ -454,7 +454,7 @@ if [[ " ${TP_ARCHIVES[*]} " =~ " BRPC " ]]; then if [[ ! -f "${PATCHED_MARK}" ]]; then for patch_file in "${TP_PATCH_DIR}"/brpc-*; do echo "patch ${patch_file}" - patch -p1 <"${patch_file}" + patch -p1 --ignore-whitespace <"${patch_file}" done touch "${PATCHED_MARK}" fi diff --git a/thirdparty/patches/brpc-1.4.0-no-pthread_mutex-hook.patch b/thirdparty/patches/brpc-1.4.0-no-pthread_mutex-hook.patch new file mode 100644 index 00000000000000..d44884a5fb3fb1 --- /dev/null +++ b/thirdparty/patches/brpc-1.4.0-no-pthread_mutex-hook.patch @@ -0,0 +1,419 @@ +diff --git a/src/brpc/builtin/hotspots_service.cpp b/src/brpc/builtin/hotspots_service.cpp +index c7577320..a5575b39 100644 +--- a/src/brpc/builtin/hotspots_service.cpp ++++ b/src/brpc/builtin/hotspots_service.cpp +@@ -760,8 +760,7 @@ static void DoProfiling(ProfilingType type, + ProfilerStop(); + } else if (type == PROFILING_CONTENTION) { + if (!bthread::ContentionProfilerStart(prof_name)) { +- os << "Another profiler (not via /hotspots/contention) is running, " +- "try again later" << (use_html ? "" : "\n"); ++ os << "Contention profiler has been disabled by doris\n"; + os.move_to(resp); + cntl->http_response().set_status_code(HTTP_STATUS_SERVICE_UNAVAILABLE); + return NotifyWaiters(type, cntl, view); +@@ -855,6 +854,12 @@ static void StartProfiling(ProfilingType type, + enabled = IsHeapProfilerEnabled(); + } + const char* const type_str = ProfilingType2String(type); ++ if (type == PROFILING_CONTENTION) { ++ os << "Error: " << type_str << " profiler is disabled by doris.\n"; ++ os.move_to(cntl->response_attachment()); ++ cntl->http_response().set_status_code(HTTP_STATUS_SERVICE_UNAVAILABLE); ++ return; ++ } + + #if defined(OS_MACOSX) + if (!has_GOOGLE_PPROF_BINARY_PATH()) { +diff --git a/src/brpc/builtin/pprof_service.cpp b/src/brpc/builtin/pprof_service.cpp +index eba71377..48fa5560 100644 +--- a/src/brpc/builtin/pprof_service.cpp ++++ b/src/brpc/builtin/pprof_service.cpp +@@ -188,7 +188,7 @@ void PProfService::contention( + return; + } + if (!bthread::ContentionProfilerStart(prof_name)) { +- cntl->SetFailed(EAGAIN, "Another profiler is running, try again later"); ++ cntl->SetFailed(EAGAIN, "Contention profiler has been disabled by doris"); + return; + } + if (bthread_usleep(sleep_sec * 1000000L) != 0) { +diff --git a/src/bthread/mutex.cpp b/src/bthread/mutex.cpp +index 3d38ef93..aad189bc 100644 +--- a/src/bthread/mutex.cpp ++++ b/src/bthread/mutex.cpp +@@ -305,6 +305,8 @@ static int64_t get_nconflicthash(void*) { + + // Start profiling contention. + bool ContentionProfilerStart(const char* filename) { ++ LOG(ERROR) << "Contention profiler is disabled by doris."; ++ return false; + if (filename == NULL) { + LOG(ERROR) << "Parameter [filename] is NULL"; + return false; +@@ -364,77 +366,79 @@ make_contention_site_invalid(bthread_contention_site_t* cs) { + cs->sampling_range = 0; + } + +-// Replace pthread_mutex_lock and pthread_mutex_unlock: +-// First call to sys_pthread_mutex_lock sets sys_pthread_mutex_lock to the +-// real function so that next calls go to the real function directly. This +-// technique avoids calling pthread_once each time. +-typedef int (*MutexOp)(pthread_mutex_t*); +-int first_sys_pthread_mutex_lock(pthread_mutex_t* mutex); +-int first_sys_pthread_mutex_unlock(pthread_mutex_t* mutex); +-static MutexOp sys_pthread_mutex_lock = first_sys_pthread_mutex_lock; +-static MutexOp sys_pthread_mutex_unlock = first_sys_pthread_mutex_unlock; +-static pthread_once_t init_sys_mutex_lock_once = PTHREAD_ONCE_INIT; +- +-// dlsym may call malloc to allocate space for dlerror and causes contention +-// profiler to deadlock at boostraping when the program is linked with +-// libunwind. The deadlock bt: +-// #0 0x00007effddc99b80 in __nanosleep_nocancel () at ../sysdeps/unix/syscall-template.S:81 +-// #1 0x00000000004b4df7 in butil::internal::SpinLockDelay(int volatile*, int, int) () +-// #2 0x00000000004b4d57 in SpinLock::SlowLock() () +-// #3 0x00000000004b4a63 in tcmalloc::ThreadCache::InitModule() () +-// #4 0x00000000004aa2b5 in tcmalloc::ThreadCache::GetCache() () +-// #5 0x000000000040c6c5 in (anonymous namespace)::do_malloc_no_errno(unsigned long) [clone.part.16] () +-// #6 0x00000000006fc125 in tc_calloc () +-// #7 0x00007effdd245690 in _dlerror_run (operate=operate@entry=0x7effdd245130 , args=args@entry=0x7fff483dedf0) at dlerror.c:141 +-// #8 0x00007effdd245198 in __dlsym (handle=, name=) at dlsym.c:70 +-// #9 0x0000000000666517 in bthread::init_sys_mutex_lock () at bthread/mutex.cpp:358 +-// #10 0x00007effddc97a90 in pthread_once () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_once.S:103 +-// #11 0x000000000066649f in bthread::first_sys_pthread_mutex_lock (mutex=0xbaf880 <_ULx86_64_lock>) at bthread/mutex.cpp:366 +-// #12 0x00000000006678bc in pthread_mutex_lock_impl (mutex=0xbaf880 <_ULx86_64_lock>) at bthread/mutex.cpp:489 +-// #13 pthread_mutex_lock (__mutex=__mutex@entry=0xbaf880 <_ULx86_64_lock>) at bthread/mutex.cpp:751 +-// #14 0x00000000004c6ea1 in _ULx86_64_init () at x86_64/Gglobal.c:83 +-// #15 0x00000000004c44fb in _ULx86_64_init_local (cursor=0x7fff483df340, uc=0x7fff483def90) at x86_64/Ginit_local.c:47 +-// #16 0x00000000004b5012 in GetStackTrace(void**, int, int) () +-// #17 0x00000000004b2095 in tcmalloc::PageHeap::GrowHeap(unsigned long) () +-// #18 0x00000000004b23a3 in tcmalloc::PageHeap::New(unsigned long) () +-// #19 0x00000000004ad457 in tcmalloc::CentralFreeList::Populate() () +-// #20 0x00000000004ad628 in tcmalloc::CentralFreeList::FetchFromSpansSafe() () +-// #21 0x00000000004ad6a3 in tcmalloc::CentralFreeList::RemoveRange(void**, void**, int) () +-// #22 0x00000000004b3ed3 in tcmalloc::ThreadCache::FetchFromCentralCache(unsigned long, unsigned long) () +-// #23 0x00000000006fbb9a in tc_malloc () +-// Call _dl_sym which is a private function in glibc to workaround the malloc +-// causing deadlock temporarily. This fix is hardly portable. +- +-static void init_sys_mutex_lock() { +-#if defined(OS_LINUX) +- // TODO: may need dlvsym when GLIBC has multiple versions of a same symbol. +- // http://blog.fesnel.com/blog/2009/08/25/preloading-with-multiple-symbol-versions +- if (_dl_sym) { +- sys_pthread_mutex_lock = (MutexOp)_dl_sym(RTLD_NEXT, "pthread_mutex_lock", (void*)init_sys_mutex_lock); +- sys_pthread_mutex_unlock = (MutexOp)_dl_sym(RTLD_NEXT, "pthread_mutex_unlock", (void*)init_sys_mutex_lock); +- } else { +- // _dl_sym may be undefined reference in some system, fallback to dlsym +- sys_pthread_mutex_lock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_lock"); +- sys_pthread_mutex_unlock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_unlock"); +- } +-#elif defined(OS_MACOSX) +- // TODO: look workaround for dlsym on mac +- sys_pthread_mutex_lock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_lock"); +- sys_pthread_mutex_unlock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_unlock"); +-#endif +-} +- +-// Make sure pthread functions are ready before main(). +-const int ALLOW_UNUSED dummy = pthread_once(&init_sys_mutex_lock_once, init_sys_mutex_lock); +- +-int first_sys_pthread_mutex_lock(pthread_mutex_t* mutex) { +- pthread_once(&init_sys_mutex_lock_once, init_sys_mutex_lock); +- return sys_pthread_mutex_lock(mutex); +-} +-int first_sys_pthread_mutex_unlock(pthread_mutex_t* mutex) { +- pthread_once(&init_sys_mutex_lock_once, init_sys_mutex_lock); +- return sys_pthread_mutex_unlock(mutex); +-} ++// #ifndef NO_PTHREAD_MUTEX_HOOK ++// // Replace pthread_mutex_lock and pthread_mutex_unlock: ++// // First call to sys_pthread_mutex_lock sets sys_pthread_mutex_lock to the ++// // real function so that next calls go to the real function directly. This ++// // technique avoids calling pthread_once each time. ++// typedef int (*MutexOp)(pthread_mutex_t*); ++// int first_sys_pthread_mutex_lock(pthread_mutex_t* mutex); ++// int first_sys_pthread_mutex_unlock(pthread_mutex_t* mutex); ++// static MutexOp sys_pthread_mutex_lock = first_sys_pthread_mutex_lock; ++// static MutexOp sys_pthread_mutex_unlock = first_sys_pthread_mutex_unlock; ++// static pthread_once_t init_sys_mutex_lock_once = PTHREAD_ONCE_INIT; ++ ++// // dlsym may call malloc to allocate space for dlerror and causes contention ++// // profiler to deadlock at boostraping when the program is linked with ++// // libunwind. The deadlock bt: ++// // #0 0x00007effddc99b80 in __nanosleep_nocancel () at ../sysdeps/unix/syscall-template.S:81 ++// // #1 0x00000000004b4df7 in butil::internal::SpinLockDelay(int volatile*, int, int) () ++// // #2 0x00000000004b4d57 in SpinLock::SlowLock() () ++// // #3 0x00000000004b4a63 in tcmalloc::ThreadCache::InitModule() () ++// // #4 0x00000000004aa2b5 in tcmalloc::ThreadCache::GetCache() () ++// // #5 0x000000000040c6c5 in (anonymous namespace)::do_malloc_no_errno(unsigned long) [clone.part.16] () ++// // #6 0x00000000006fc125 in tc_calloc () ++// // #7 0x00007effdd245690 in _dlerror_run (operate=operate@entry=0x7effdd245130 , args=args@entry=0x7fff483dedf0) at dlerror.c:141 ++// // #8 0x00007effdd245198 in __dlsym (handle=, name=) at dlsym.c:70 ++// // #9 0x0000000000666517 in bthread::init_sys_mutex_lock () at bthread/mutex.cpp:358 ++// // #10 0x00007effddc97a90 in pthread_once () at ../nptl/sysdeps/unix/sysv/linux/x86_64/pthread_once.S:103 ++// // #11 0x000000000066649f in bthread::first_sys_pthread_mutex_lock (mutex=0xbaf880 <_ULx86_64_lock>) at bthread/mutex.cpp:366 ++// // #12 0x00000000006678bc in pthread_mutex_lock_impl (mutex=0xbaf880 <_ULx86_64_lock>) at bthread/mutex.cpp:489 ++// // #13 pthread_mutex_lock (__mutex=__mutex@entry=0xbaf880 <_ULx86_64_lock>) at bthread/mutex.cpp:751 ++// // #14 0x00000000004c6ea1 in _ULx86_64_init () at x86_64/Gglobal.c:83 ++// // #15 0x00000000004c44fb in _ULx86_64_init_local (cursor=0x7fff483df340, uc=0x7fff483def90) at x86_64/Ginit_local.c:47 ++// // #16 0x00000000004b5012 in GetStackTrace(void**, int, int) () ++// // #17 0x00000000004b2095 in tcmalloc::PageHeap::GrowHeap(unsigned long) () ++// // #18 0x00000000004b23a3 in tcmalloc::PageHeap::New(unsigned long) () ++// // #19 0x00000000004ad457 in tcmalloc::CentralFreeList::Populate() () ++// // #20 0x00000000004ad628 in tcmalloc::CentralFreeList::FetchFromSpansSafe() () ++// // #21 0x00000000004ad6a3 in tcmalloc::CentralFreeList::RemoveRange(void**, void**, int) () ++// // #22 0x00000000004b3ed3 in tcmalloc::ThreadCache::FetchFromCentralCache(unsigned long, unsigned long) () ++// // #23 0x00000000006fbb9a in tc_malloc () ++// // Call _dl_sym which is a private function in glibc to workaround the malloc ++// // causing deadlock temporarily. This fix is hardly portable. ++ ++// static void init_sys_mutex_lock() { ++// #if defined(OS_LINUX) ++// // TODO: may need dlvsym when GLIBC has multiple versions of a same symbol. ++// // http://blog.fesnel.com/blog/2009/08/25/preloading-with-multiple-symbol-versions ++// if (_dl_sym) { ++// sys_pthread_mutex_lock = (MutexOp)_dl_sym(RTLD_NEXT, "pthread_mutex_lock", (void*)init_sys_mutex_lock); ++// sys_pthread_mutex_unlock = (MutexOp)_dl_sym(RTLD_NEXT, "pthread_mutex_unlock", (void*)init_sys_mutex_lock); ++// } else { ++// // _dl_sym may be undefined reference in some system, fallback to dlsym ++// sys_pthread_mutex_lock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_lock"); ++// sys_pthread_mutex_unlock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_unlock"); ++// } ++// #elif defined(OS_MACOSX) ++// // TODO: look workaround for dlsym on mac ++// sys_pthread_mutex_lock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_lock"); ++// sys_pthread_mutex_unlock = (MutexOp)dlsym(RTLD_NEXT, "pthread_mutex_unlock"); ++// #endif ++// } ++ ++// // Make sure pthread functions are ready before main(). ++// const int ALLOW_UNUSED dummy = pthread_once(&init_sys_mutex_lock_once, init_sys_mutex_lock); ++ ++// int first_sys_pthread_mutex_lock(pthread_mutex_t* mutex) { ++// pthread_once(&init_sys_mutex_lock_once, init_sys_mutex_lock); ++// return sys_pthread_mutex_lock(mutex); ++// } ++// int first_sys_pthread_mutex_unlock(pthread_mutex_t* mutex) { ++// pthread_once(&init_sys_mutex_lock_once, init_sys_mutex_lock); ++// return sys_pthread_mutex_unlock(mutex); ++// } ++// #endif + + inline uint64_t hash_mutex_ptr(const pthread_mutex_t* m) { + return butil::fmix64((uint64_t)m); +@@ -524,99 +528,101 @@ void submit_contention(const bthread_contention_site_t& csite, int64_t now_ns) { + tls_inside_lock = false; + } + +-BUTIL_FORCE_INLINE int pthread_mutex_lock_impl(pthread_mutex_t* mutex) { +- // Don't change behavior of lock when profiler is off. +- if (!g_cp || +- // collecting code including backtrace() and submit() may call +- // pthread_mutex_lock and cause deadlock. Don't sample. +- tls_inside_lock) { +- return sys_pthread_mutex_lock(mutex); +- } +- // Don't slow down non-contended locks. +- int rc = pthread_mutex_trylock(mutex); +- if (rc != EBUSY) { +- return rc; +- } +- // Ask bvar::Collector if this (contended) locking should be sampled +- const size_t sampling_range = bvar::is_collectable(&g_cp_sl); +- +- bthread_contention_site_t* csite = NULL; +-#ifndef DONT_SPEEDUP_PTHREAD_CONTENTION_PROFILER_WITH_TLS +- TLSPthreadContentionSites& fast_alt = tls_csites; +- if (fast_alt.cp_version != g_cp_version) { +- fast_alt.cp_version = g_cp_version; +- fast_alt.count = 0; +- } +- if (fast_alt.count < TLS_MAX_COUNT) { +- MutexAndContentionSite& entry = fast_alt.list[fast_alt.count++]; +- entry.mutex = mutex; +- csite = &entry.csite; +- if (!sampling_range) { +- make_contention_site_invalid(&entry.csite); +- return sys_pthread_mutex_lock(mutex); +- } +- } +-#endif +- if (!sampling_range) { // don't sample +- return sys_pthread_mutex_lock(mutex); +- } +- // Lock and monitor the waiting time. +- const int64_t start_ns = butil::cpuwide_time_ns(); +- rc = sys_pthread_mutex_lock(mutex); +- if (!rc) { // Inside lock +- if (!csite) { +- csite = add_pthread_contention_site(mutex); +- if (csite == NULL) { +- return rc; +- } +- } +- csite->duration_ns = butil::cpuwide_time_ns() - start_ns; +- csite->sampling_range = sampling_range; +- } // else rare +- return rc; +-} +- +-BUTIL_FORCE_INLINE int pthread_mutex_unlock_impl(pthread_mutex_t* mutex) { +- // Don't change behavior of unlock when profiler is off. +- if (!g_cp || tls_inside_lock) { +- // This branch brings an issue that an entry created by +- // add_pthread_contention_site may not be cleared. Thus we add a +- // 16-bit rolling version in the entry to find out such entry. +- return sys_pthread_mutex_unlock(mutex); +- } +- int64_t unlock_start_ns = 0; +- bool miss_in_tls = true; +- bthread_contention_site_t saved_csite = {0,0}; +-#ifndef DONT_SPEEDUP_PTHREAD_CONTENTION_PROFILER_WITH_TLS +- TLSPthreadContentionSites& fast_alt = tls_csites; +- for (int i = fast_alt.count - 1; i >= 0; --i) { +- if (fast_alt.list[i].mutex == mutex) { +- if (is_contention_site_valid(fast_alt.list[i].csite)) { +- saved_csite = fast_alt.list[i].csite; +- unlock_start_ns = butil::cpuwide_time_ns(); +- } +- fast_alt.list[i] = fast_alt.list[--fast_alt.count]; +- miss_in_tls = false; +- break; +- } +- } +-#endif +- // Check the map to see if the lock is sampled. Notice that we're still +- // inside critical section. +- if (miss_in_tls) { +- if (remove_pthread_contention_site(mutex, &saved_csite)) { +- unlock_start_ns = butil::cpuwide_time_ns(); +- } +- } +- const int rc = sys_pthread_mutex_unlock(mutex); +- // [Outside lock] +- if (unlock_start_ns) { +- const int64_t unlock_end_ns = butil::cpuwide_time_ns(); +- saved_csite.duration_ns += unlock_end_ns - unlock_start_ns; +- submit_contention(saved_csite, unlock_end_ns); +- } +- return rc; +-} ++// #ifndef NO_PTHREAD_MUTEX_HOOK ++// BUTIL_FORCE_INLINE int pthread_mutex_lock_impl(pthread_mutex_t* mutex) { ++// // Don't change behavior of lock when profiler is off. ++// if (!g_cp || ++// // collecting code including backtrace() and submit() may call ++// // pthread_mutex_lock and cause deadlock. Don't sample. ++// tls_inside_lock) { ++// return sys_pthread_mutex_lock(mutex); ++// } ++// // Don't slow down non-contended locks. ++// int rc = pthread_mutex_trylock(mutex); ++// if (rc != EBUSY) { ++// return rc; ++// } ++// // Ask bvar::Collector if this (contended) locking should be sampled ++// const size_t sampling_range = bvar::is_collectable(&g_cp_sl); ++ ++// bthread_contention_site_t* csite = NULL; ++// #ifndef DONT_SPEEDUP_PTHREAD_CONTENTION_PROFILER_WITH_TLS ++// TLSPthreadContentionSites& fast_alt = tls_csites; ++// if (fast_alt.cp_version != g_cp_version) { ++// fast_alt.cp_version = g_cp_version; ++// fast_alt.count = 0; ++// } ++// if (fast_alt.count < TLS_MAX_COUNT) { ++// MutexAndContentionSite& entry = fast_alt.list[fast_alt.count++]; ++// entry.mutex = mutex; ++// csite = &entry.csite; ++// if (!sampling_range) { ++// make_contention_site_invalid(&entry.csite); ++// return sys_pthread_mutex_lock(mutex); ++// } ++// } ++// #endif ++// if (!sampling_range) { // don't sample ++// return sys_pthread_mutex_lock(mutex); ++// } ++// // Lock and monitor the waiting time. ++// const int64_t start_ns = butil::cpuwide_time_ns(); ++// rc = sys_pthread_mutex_lock(mutex); ++// if (!rc) { // Inside lock ++// if (!csite) { ++// csite = add_pthread_contention_site(mutex); ++// if (csite == NULL) { ++// return rc; ++// } ++// } ++// csite->duration_ns = butil::cpuwide_time_ns() - start_ns; ++// csite->sampling_range = sampling_range; ++// } // else rare ++// return rc; ++// } ++ ++// BUTIL_FORCE_INLINE int pthread_mutex_unlock_impl(pthread_mutex_t* mutex) { ++// // Don't change behavior of unlock when profiler is off. ++// if (!g_cp || tls_inside_lock) { ++// // This branch brings an issue that an entry created by ++// // add_pthread_contention_site may not be cleared. Thus we add a ++// // 16-bit rolling version in the entry to find out such entry. ++// return sys_pthread_mutex_unlock(mutex); ++// } ++// int64_t unlock_start_ns = 0; ++// bool miss_in_tls = true; ++// bthread_contention_site_t saved_csite = {0,0}; ++// #ifndef DONT_SPEEDUP_PTHREAD_CONTENTION_PROFILER_WITH_TLS ++// TLSPthreadContentionSites& fast_alt = tls_csites; ++// for (int i = fast_alt.count - 1; i >= 0; --i) { ++// if (fast_alt.list[i].mutex == mutex) { ++// if (is_contention_site_valid(fast_alt.list[i].csite)) { ++// saved_csite = fast_alt.list[i].csite; ++// unlock_start_ns = butil::cpuwide_time_ns(); ++// } ++// fast_alt.list[i] = fast_alt.list[--fast_alt.count]; ++// miss_in_tls = false; ++// break; ++// } ++// } ++// #endif ++// // Check the map to see if the lock is sampled. Notice that we're still ++// // inside critical section. ++// if (miss_in_tls) { ++// if (remove_pthread_contention_site(mutex, &saved_csite)) { ++// unlock_start_ns = butil::cpuwide_time_ns(); ++// } ++// } ++// const int rc = sys_pthread_mutex_unlock(mutex); ++// // [Outside lock] ++// if (unlock_start_ns) { ++// const int64_t unlock_end_ns = butil::cpuwide_time_ns(); ++// saved_csite.duration_ns += unlock_end_ns - unlock_start_ns; ++// submit_contention(saved_csite, unlock_end_ns); ++// } ++// return rc; ++// } ++// #endif + + // Implement bthread_mutex_t related functions + struct MutexInternal { +@@ -815,11 +821,13 @@ int bthread_mutex_unlock(bthread_mutex_t* m) { + return 0; + } + +-int pthread_mutex_lock (pthread_mutex_t *__mutex) { +- return bthread::pthread_mutex_lock_impl(__mutex); +-} +-int pthread_mutex_unlock (pthread_mutex_t *__mutex) { +- return bthread::pthread_mutex_unlock_impl(__mutex); +-} ++// #ifndef NO_PTHREAD_MUTEX_HOOK ++// int pthread_mutex_lock (pthread_mutex_t *__mutex) { ++// return bthread::pthread_mutex_lock_impl(__mutex); ++// } ++// int pthread_mutex_unlock (pthread_mutex_t *__mutex) { ++// return bthread::pthread_mutex_unlock_impl(__mutex); ++// } ++// #endif + + } // extern "C" diff --git a/thirdparty/patches/librdkafka-1.9.2.patch b/thirdparty/patches/librdkafka-1.9.2.patch index b13e740bc5c36a..3caac08f79dacd 100644 --- a/thirdparty/patches/librdkafka-1.9.2.patch +++ b/thirdparty/patches/librdkafka-1.9.2.patch @@ -67,7 +67,19 @@ --- src/rdkafka_broker.c +++ src/rdkafka_broker.c -@@ -5461,7 +5461,9 @@ static int rd_kafka_broker_thread_main(void *arg) { +@@ -3288,6 +3288,11 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { + : (topic_err + ? topic_err + : RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION)); ++ ++ if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER) { ++ rd_kafka_toppar_purge_internal_fetch_queue_maybe( ++ rktp); ++ } + } + + rd_kafka_toppar_unlock(rktp); +@@ -5461,7 +5466,9 @@ static int rd_kafka_broker_thread_main(void *arg) { */ void rd_kafka_broker_destroy_final(rd_kafka_broker_t *rkb) { @@ -78,3 +90,100 @@ rd_assert(TAILQ_EMPTY(&rkb->rkb_monitors)); rd_assert(TAILQ_EMPTY(&rkb->rkb_outbufs.rkbq_bufs)); rd_assert(TAILQ_EMPTY(&rkb->rkb_waitresps.rkbq_bufs)); +--- src/rdkafka_cgrp.c ++++ src/rdkafka_cgrp.c +@@ -2734,6 +2734,9 @@ static void rd_kafka_cgrp_partition_del(rd_kafka_cgrp_t *rkcg, + rd_kafka_toppar_lock(rktp); + rd_assert(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP); + rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_ON_CGRP; ++ ++ rd_kafka_toppar_purge_internal_fetch_queue_maybe(rktp); ++ + rd_kafka_toppar_unlock(rktp); + + rd_list_remove(&rkcg->rkcg_toppars, rktp); +--- src/rdkafka_partition.c ++++ src/rdkafka_partition.c +@@ -959,7 +959,71 @@ void rd_kafka_toppar_insert_msgq(rd_kafka_toppar_t *rktp, + rd_kafka_toppar_unlock(rktp); + } + ++/** ++ * @brief Purge internal fetch queue if toppar is stopped ++ * (RD_KAFKA_TOPPAR_FETCH_STOPPED) and removed from the cluster ++ * (RD_KAFKA_TOPPAR_F_REMOVE). Will be called from different places as it's ++ * removed starting from a metadata response and stopped from a rebalance or a ++ * consumer close. ++ * ++ * @remark Avoids circular dependencies in from `rktp_fetchq` ops to the same ++ * toppar that stop destroying a consumer. ++ * ++ * @locks rd_kafka_toppar_lock() MUST be held ++ */ ++void rd_kafka_toppar_purge_internal_fetch_queue_maybe(rd_kafka_toppar_t *rktp) { ++ rd_kafka_q_t *rkq; ++ rkq = rktp->rktp_fetchq; ++ mtx_lock(&rkq->rkq_lock); ++ if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_REMOVE && ++ !rktp->rktp_fetchq->rkq_fwdq) { ++ rd_kafka_op_t *rko; ++ int cnt = 0, barrier_cnt = 0, message_cnt = 0, other_cnt = 0; ++ ++ /* Partition is being removed from the cluster and it's stopped, ++ * so rktp->rktp_fetchq->rkq_fwdq is NULL. ++ * Purge remaining operations in rktp->rktp_fetchq->rkq_q, ++ * while holding lock, to avoid circular references */ ++ rko = TAILQ_FIRST(&rkq->rkq_q); ++ while (rko) { ++ if (rko->rko_type != RD_KAFKA_OP_BARRIER && ++ rko->rko_type != RD_KAFKA_OP_FETCH) { ++ rd_kafka_log( ++ rktp->rktp_rkt->rkt_rk, LOG_WARNING, ++ "PARTDEL", ++ "Purging toppar fetch queue buffer op" ++ "with unexpected type: %s", ++ rd_kafka_op2str(rko->rko_type)); ++ } ++ ++ if (rko->rko_type == RD_KAFKA_OP_BARRIER) ++ barrier_cnt++; ++ else if (rko->rko_type == RD_KAFKA_OP_FETCH) ++ message_cnt++; ++ else ++ other_cnt++; + ++ rko = TAILQ_NEXT(rko, rko_link); ++ cnt++; ++ } ++ ++ if (cnt) { ++ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, CGRP, "PARTDEL", ++ "Purge toppar fetch queue buffer " ++ "containing %d op(s) " ++ "(%d barrier(s), %d message(s), %d other)" ++ " to avoid " ++ "circular references", ++ cnt, barrier_cnt, message_cnt, other_cnt); ++ rd_kafka_q_purge0(rktp->rktp_fetchq, rd_false); ++ } else { ++ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, CGRP, "PARTDEL", ++ "Not purging toppar fetch queue buffer." ++ " No ops present in the buffer."); ++ } ++ } ++ mtx_unlock(&rkq->rkq_lock); ++} + + /** + * Helper method for purging queues when removing a toppar. +--- src/rdkafka_partition.h ++++ src/rdkafka_partition.h +@@ -541,6 +541,8 @@ void rd_kafka_toppar_offset_request(rd_kafka_toppar_t *rktp, + int64_t query_offset, + int backoff_ms); + ++void rd_kafka_toppar_purge_internal_fetch_queue_maybe(rd_kafka_toppar_t *rktp); ++ + int rd_kafka_toppar_purge_queues(rd_kafka_toppar_t *rktp, + int purge_flags, + rd_bool_t include_xmit_msgq);