diff --git a/.asf.yaml b/.asf.yaml index 3e6c75a1b360b47..a82497c1078c61c 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -88,6 +88,7 @@ github: - nanfeng1999 - gitccl - shuke987 + - wm1581066 notifications: pullrequests_status: commits@doris.apache.org diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 389a23d77b5d3db..926d4fbb99073c8 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1088,6 +1088,8 @@ DEFINE_Int16(bitmap_serialize_version, "1"); // the count of thread to group commit insert DEFINE_Int32(group_commit_insert_threads, "10"); +DEFINE_mInt32(scan_thread_nice_value, "0"); + #ifdef BE_TEST // test s3 DEFINE_String(test_s3_resource, "resource"); diff --git a/be/src/common/config.h b/be/src/common/config.h index b34a6c98ccda5b7..2a6ed63125d99e7 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1159,6 +1159,12 @@ DECLARE_Int16(bitmap_serialize_version); // This config can be set to limit thread number in group commit insert thread pool. DECLARE_mInt32(group_commit_insert_threads); +// The configuration item is used to lower the priority of the scanner thread, +// typically employed to ensure CPU scheduling for write operations. +// Default is 0, which is default value of thread nice value, increase this value +// to lower the priority of scan threads +DECLARE_Int32(scan_thread_nice_value); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/common/status.cpp b/be/src/common/status.cpp index ec76755fce35db5..bc8ed22a235eddf 100644 --- a/be/src/common/status.cpp +++ b/be/src/common/status.cpp @@ -39,7 +39,7 @@ TStatus Status::to_thrift() const { void Status::to_protobuf(PStatus* s) const { s->clear_error_msgs(); s->set_status_code((int)_code); - if (!ok() && _err_msg) { + if (!ok()) { s->add_error_msgs(fmt::format("({})[{}]{}", BackendOptions::get_localhost(), code_as_string(), _err_msg ? _err_msg->_msg : "")); } diff --git a/be/src/common/status.h b/be/src/common/status.h index 465a90ab4cd3ca1..52c43829a2a33d1 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -350,14 +350,18 @@ class Status { return *this; } + template Status static create(const TStatus& status) { - return Error(status.status_code, - status.error_msgs.empty() ? "" : status.error_msgs[0]); + return Error( + status.status_code, + "TStatus: " + (status.error_msgs.empty() ? "" : status.error_msgs[0])); } + template Status static create(const PStatus& pstatus) { - return Error(pstatus.status_code(), - pstatus.error_msgs_size() == 0 ? "" : pstatus.error_msgs(0)); + return Error( + pstatus.status_code(), + "PStatus: " + (pstatus.error_msgs_size() == 0 ? "" : pstatus.error_msgs(0))); } template diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index fdcf5a76c588592..f62fda89a0ac58e 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -134,6 +134,7 @@ Status Compaction::do_compaction(int64_t permits) { << ", before=" << checksum_before << ", checksum_after=" << checksum_after; } } + _load_segment_to_cache(); return st; } @@ -825,6 +826,14 @@ int64_t Compaction::get_compaction_permits() { return permits; } +Status Compaction::_load_segment_to_cache() { + // Load new rowset's segments to cache. + SegmentCacheHandle handle; + RETURN_IF_ERROR(SegmentLoader::instance()->load_segments( + std::static_pointer_cast(_output_rowset), &handle, true)); + return Status::OK(); +} + #ifdef BE_TEST void Compaction::set_input_rowset(const std::vector& rowsets) { _input_rowsets = rowsets; diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index 859615c0361d56d..afb4f27e7ffe7bb 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -99,6 +99,7 @@ class Compaction { private: bool _check_if_includes_input_rowsets(const RowsetIdUnorderedSet& commit_rowset_ids_set) const; + Status _load_segment_to_cache(); protected: // the root tracker for this compaction diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index 50eb1e85b4c60fd..b3a2a6c76ce7d69 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -447,7 +447,9 @@ class AnalyticDependency final : public WriteDependency { auto need_more_input = whether_need_next_partition(_analytic_state.found_partition_end); if (need_more_input) { block_reading(); + set_ready_for_write(); } else { + block_writing(); set_ready_for_read(); } return need_more_input; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 37732e9036918bd..5a0d30f0efaa2d6 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -139,7 +139,7 @@ FragmentMgr::FragmentMgr(ExecEnv* exec_env) CHECK(s.ok()) << s.to_string(); } -FragmentMgr::~FragmentMgr() {} +FragmentMgr::~FragmentMgr() = default; void FragmentMgr::stop() { DEREGISTER_HOOK_METRIC(plan_fragment_count); @@ -185,9 +185,9 @@ void FragmentMgr::coordinator_callback(const ReportStatusRequest& req) { if (!coord_status.ok()) { std::stringstream ss; UniqueId uid(req.query_id.hi, req.query_id.lo); - ss << "couldn't get a client for " << req.coord_addr << ", reason: " << coord_status; - LOG(WARNING) << "query_id: " << uid << ", " << ss.str(); - req.update_fn(Status::InternalError(ss.str())); + req.update_fn(Status::InternalError( + "query_id: {}, couldn't get a client for {}, reason is {}", uid.to_string(), + PrintThriftNetworkAddress(req.coord_addr), coord_status.to_string())); return; } @@ -322,12 +322,10 @@ void FragmentMgr::coordinator_callback(const ReportStatusRequest& req) { coord->reportExecStatus(res, params); } - rpc_status = Status(Status::create(res.status)); + rpc_status = Status::create(res.status); } catch (TException& e) { - std::stringstream msg; - msg << "ReportExecStatus() to " << req.coord_addr << " failed:\n" << e.what(); - LOG(WARNING) << msg.str(); - rpc_status = Status::InternalError(msg.str()); + rpc_status = Status::InternalError("ReportExecStatus() to {} failed: {}", + PrintThriftNetworkAddress(req.coord_addr), e.what()); } if (!rpc_status.ok()) { @@ -1232,7 +1230,7 @@ Status FragmentMgr::apply_filterv2(const PPublishFilterRequestV2* request, std::shared_ptr pip_context; RuntimeFilterMgr* runtime_filter_mgr = nullptr; - ObjectPool* pool; + ObjectPool* pool = nullptr; if (is_pipeline) { std::unique_lock lock(_lock); auto iter = _pipeline_map.find(tfragment_instance_id); diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index b537b018b19e1f2..98064cf9d94a539 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -401,9 +401,7 @@ void BackendService::ingest_binlog(TIngestBinlogResult& result, }; if (!config::enable_feature_binlog) { - auto error_msg = "enable feature binlog is false"; - LOG(WARNING) << error_msg; - set_tstatus(TStatusCode::RUNTIME_ERROR, error_msg); + set_tstatus(TStatusCode::RUNTIME_ERROR, "enable feature binlog is false"); return; } diff --git a/be/src/util/sha.cpp b/be/src/util/sha.cpp new file mode 100644 index 000000000000000..68099ff9269bbd1 --- /dev/null +++ b/be/src/util/sha.cpp @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT 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 "util/sha.h" + +#include + +#include + +namespace doris { + +constexpr static char dig_vec_lower[] = "0123456789abcdef"; + +void SHA1Digest::reset(const void* data, size_t length) { + SHA1_Init(&_sha_ctx); + SHA1_Update(&_sha_ctx, data, length); +} + +std::string_view SHA1Digest::digest() { + unsigned char buf[SHA_DIGEST_LENGTH]; + SHA1_Final(buf, &_sha_ctx); + + char* to = _reuse_hex; + for (int i = 0; i < SHA_DIGEST_LENGTH; ++i) { + *to++ = dig_vec_lower[buf[i] >> 4]; + *to++ = dig_vec_lower[buf[i] & 0x0F]; + } + + return std::string_view {_reuse_hex, _reuse_hex + 2 * SHA_DIGEST_LENGTH}; +} + +void SHA224Digest::reset(const void* data, size_t length) { + SHA224_Init(&_sha224_ctx); + SHA224_Update(&_sha224_ctx, data, length); +} + +std::string_view SHA224Digest::digest() { + unsigned char buf[SHA224_DIGEST_LENGTH]; + SHA224_Final(buf, &_sha224_ctx); + + char* to = _reuse_hex; + for (int i = 0; i < SHA224_DIGEST_LENGTH; ++i) { + *to++ = dig_vec_lower[buf[i] >> 4]; + *to++ = dig_vec_lower[buf[i] & 0x0F]; + } + + return std::string_view {_reuse_hex, _reuse_hex + 2 * SHA224_DIGEST_LENGTH}; +} + +void SHA256Digest::reset(const void* data, size_t length) { + SHA256_Init(&_sha256_ctx); + SHA256_Update(&_sha256_ctx, data, length); +} + +std::string_view SHA256Digest::digest() { + unsigned char buf[SHA256_DIGEST_LENGTH]; + SHA256_Final(buf, &_sha256_ctx); + + char* to = _reuse_hex; + for (int i = 0; i < SHA256_DIGEST_LENGTH; ++i) { + *to++ = dig_vec_lower[buf[i] >> 4]; + *to++ = dig_vec_lower[buf[i] & 0x0F]; + } + + return std::string_view {_reuse_hex, _reuse_hex + 2 * SHA256_DIGEST_LENGTH}; +} + +void SHA384Digest::reset(const void* data, size_t length) { + SHA384_Init(&_sha384_ctx); + SHA384_Update(&_sha384_ctx, data, length); +} + +std::string_view SHA384Digest::digest() { + unsigned char buf[SHA384_DIGEST_LENGTH]; + SHA384_Final(buf, &_sha384_ctx); + + char* to = _reuse_hex; + for (int i = 0; i < SHA384_DIGEST_LENGTH; ++i) { + *to++ = dig_vec_lower[buf[i] >> 4]; + *to++ = dig_vec_lower[buf[i] & 0x0F]; + } + + return std::string_view {_reuse_hex, _reuse_hex + 2 * SHA384_DIGEST_LENGTH}; +} + +void SHA512Digest::reset(const void* data, size_t length) { + SHA512_Init(&_sha512_ctx); + SHA512_Update(&_sha512_ctx, data, length); +} + +std::string_view SHA512Digest::digest() { + unsigned char buf[SHA512_DIGEST_LENGTH]; + SHA512_Final(buf, &_sha512_ctx); + + char* to = _reuse_hex; + for (int i = 0; i < SHA512_DIGEST_LENGTH; ++i) { + *to++ = dig_vec_lower[buf[i] >> 4]; + *to++ = dig_vec_lower[buf[i] & 0x0F]; + } + + return std::string_view {_reuse_hex, _reuse_hex + 2 * SHA512_DIGEST_LENGTH}; +} + +} // namespace doris diff --git a/be/src/util/sha.h b/be/src/util/sha.h new file mode 100644 index 000000000000000..1545dd3c0316c8a --- /dev/null +++ b/be/src/util/sha.h @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include + +namespace doris { + +class SHA1Digest { +public: + void reset(const void* data, size_t length); + std::string_view digest(); + +private: + SHA_CTX _sha_ctx; + char _reuse_hex[2 * SHA_DIGEST_LENGTH]; +}; + +class SHA224Digest { +public: + void reset(const void* data, size_t length); + std::string_view digest(); + +private: + SHA256_CTX _sha224_ctx; + char _reuse_hex[2 * SHA224_DIGEST_LENGTH]; +}; + +class SHA256Digest { +public: + void reset(const void* data, size_t length); + std::string_view digest(); + +private: + SHA256_CTX _sha256_ctx; + char _reuse_hex[2 * SHA256_DIGEST_LENGTH]; +}; + +class SHA384Digest { +public: + void reset(const void* data, size_t length); + std::string_view digest(); + +private: + SHA512_CTX _sha384_ctx; + char _reuse_hex[2 * SHA384_DIGEST_LENGTH]; +}; + +class SHA512Digest { +public: + void reset(const void* data, size_t length); + std::string_view digest(); + +private: + SHA512_CTX _sha512_ctx; + char _reuse_hex[2 * SHA512_DIGEST_LENGTH]; +}; +} // namespace doris diff --git a/be/src/util/thread.cpp b/be/src/util/thread.cpp index 76b38195b3d441d..c0f3fd29198f7db 100644 --- a/be/src/util/thread.cpp +++ b/be/src/util/thread.cpp @@ -20,6 +20,8 @@ #include "thread.h" +#include + #ifndef __APPLE__ // IWYU pragma: no_include #include @@ -49,6 +51,7 @@ #include #include +#include "common/config.h" #include "common/logging.h" #include "gutil/atomicops.h" #include "gutil/dynamic_annotations.h" @@ -93,6 +96,8 @@ class ThreadMgr { #ifndef __APPLE__ static void set_idle_sched(int64_t tid); + + static void set_thread_nice_value(int64_t tid); #endif // not the system TID, since pthread_t is less prone to being recycled. @@ -174,6 +179,26 @@ void ThreadMgr::set_idle_sched(int64_t tid) { LOG(ERROR) << "set_thread_idle_sched"; } } + +void ThreadMgr::set_thread_nice_value(int64_t tid) { + if (tid == getpid()) { + return; + } + // From Linux kernel: + // In the current implementation, each unit of difference in the nice values of two + // processes results in a factor of 1.25 in the degree to which the + // scheduler favors the higher priority process. This causes very + // low nice values (+19) to truly provide little CPU to a process + // whenever there is any other higher priority load on the system, + // and makes high nice values (-20) deliver most of the CPU to + // applications that require it (e.g., some audio applications). + + // Choose 5 as lower priority value, default is 0 + int err = setpriority(PRIO_PROCESS, 0, config::scan_thread_nice_value); + if (err < 0 && errno != EPERM) { + LOG(ERROR) << "set_thread_low_priority"; + } +} #endif void ThreadMgr::add_thread(const pthread_t& pthread_id, const std::string& name, @@ -305,6 +330,10 @@ void Thread::set_self_name(const std::string& name) { void Thread::set_idle_sched() { ThreadMgr::set_idle_sched(current_thread_id()); } + +void Thread::set_thread_nice_value() { + ThreadMgr::set_thread_nice_value(current_thread_id()); +} #endif void Thread::join() { diff --git a/be/src/util/thread.h b/be/src/util/thread.h index 9514e4ef7612b41..ea9d3161d05cb8e 100644 --- a/be/src/util/thread.h +++ b/be/src/util/thread.h @@ -93,6 +93,8 @@ class Thread : public RefCountedThreadSafe { #ifndef __APPLE__ static void set_idle_sched(); + + static void set_thread_nice_value(); #endif ~Thread(); diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 3f2d591cef27c99..d3370925e995b30 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -318,6 +318,14 @@ void ScannerScheduler::_scanner_scan(ScannerScheduler* scheduler, ScannerContext Thread::set_self_name("_scanner_scan"); } #endif + +#ifndef __APPLE__ + // The configuration item is used to lower the priority of the scanner thread, + // typically employed to ensure CPU scheduling for write operations. + if (config::scan_thread_nice_value != 0 && scanner->get_name() != VFileScanner::NAME) { + Thread::set_thread_nice_value(); + } +#endif scanner->update_wait_worker_timer(); scanner->start_scan_cpu_timer(); Status status = Status::OK(); diff --git a/be/src/vec/functions/function_string.cpp b/be/src/vec/functions/function_string.cpp index 5812c7badbff23e..df9f28a57987e9e 100644 --- a/be/src/vec/functions/function_string.cpp +++ b/be/src/vec/functions/function_string.cpp @@ -972,7 +972,6 @@ 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(); @@ -980,7 +979,10 @@ 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>(); @@ -993,9 +995,10 @@ void register_function_string(SimpleFunctionFactory& factory) { factory.register_alias(SubstringUtil::name, "substr"); factory.register_alias(FunctionToLower::name, "lcase"); factory.register_alias(FunctionToUpper::name, "ucase"); - factory.register_alias(FunctionStringMd5AndSM3::name, "md5"); + factory.register_alias(FunctionStringDigestOneArg::name, "md5"); factory.register_alias(FunctionStringUTF8Length::name, "character_length"); - factory.register_alias(FunctionStringMd5AndSM3::name, "sm3"); + factory.register_alias(FunctionStringDigestOneArg::name, "sm3"); + factory.register_alias(FunctionStringDigestSHA1::name, "sha"); /// @TEMPORARY: for be_exec_version=2 factory.register_alternative_function(); diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h index 32e373ffa05b4d4..37a21a3ea5b6bc1 100644 --- a/be/src/vec/functions/function_string.h +++ b/be/src/vec/functions/function_string.h @@ -41,6 +41,7 @@ #include "runtime/decimalv2_value.h" #include "runtime/runtime_state.h" #include "runtime/string_search.hpp" +#include "util/sha.h" #include "util/string_util.h" #include "util/utf8_check.h" #include "vec/aggregate_functions/aggregate_function.h" @@ -1980,10 +1981,10 @@ struct MD5Sum { }; template -class FunctionStringMd5AndSM3 : public IFunction { +class FunctionStringDigestOneArg : public IFunction { public: static constexpr auto name = Impl::name; - static FunctionPtr create() { return std::make_shared(); } + static FunctionPtr create() { return std::make_shared(); } String get_name() const override { return name; } size_t get_number_of_arguments() const override { return 0; } bool is_variadic() const override { return true; } @@ -1991,7 +1992,6 @@ class FunctionStringMd5AndSM3 : public IFunction { DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { return std::make_shared(); } - bool use_default_implementation_for_nulls() const override { return true; } Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) override { @@ -2045,6 +2045,107 @@ class FunctionStringMd5AndSM3 : public IFunction { } }; +class FunctionStringDigestSHA1 : public IFunction { +public: + static constexpr auto name = "sha1"; + static FunctionPtr create() { return std::make_shared(); } + String get_name() const override { return name; } + size_t get_number_of_arguments() const override { return 1; } + bool is_variadic() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return std::make_shared(); + } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) override { + DCHECK_EQ(arguments.size(), 1); + + ColumnPtr str_col = block.get_by_position(arguments[0]).column; + auto& data = assert_cast(str_col.get())->get_chars(); + auto& offset = assert_cast(str_col.get())->get_offsets(); + + auto res_col = ColumnString::create(); + auto& res_data = res_col->get_chars(); + auto& res_offset = res_col->get_offsets(); + res_offset.resize(input_rows_count); + + SHA1Digest digest; + for (size_t i = 0; i < input_rows_count; ++i) { + int size = offset[i] - offset[i - 1]; + digest.reset(&data[offset[i - 1]], size); + std::string_view ans = digest.digest(); + + StringOP::push_value_string(ans, i, res_data, res_offset); + } + + block.replace_by_position(result, std::move(res_col)); + return Status::OK(); + } +}; + +class FunctionStringDigestSHA2 : public IFunction { +public: + static constexpr auto name = "sha2"; + static FunctionPtr create() { return std::make_shared(); } + String get_name() const override { return name; } + size_t get_number_of_arguments() const override { return 2; } + bool is_variadic() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return std::make_shared(); + } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) override { + DCHECK(!is_column_const(*block.get_by_position(arguments[0]).column)); + + ColumnPtr str_col = block.get_by_position(arguments[0]).column; + auto& data = assert_cast(str_col.get())->get_chars(); + auto& offset = assert_cast(str_col.get())->get_offsets(); + + [[maybe_unused]] const auto& [right_column, right_const] = + unpack_if_const(block.get_by_position(arguments[1]).column); + auto digest_length = assert_cast(right_column.get())->get_data()[0]; + + auto res_col = ColumnString::create(); + auto& res_data = res_col->get_chars(); + auto& res_offset = res_col->get_offsets(); + res_offset.resize(input_rows_count); + + if (digest_length == 224) { + execute_base(data, offset, input_rows_count, res_data, res_offset); + } else if (digest_length == 256) { + execute_base(data, offset, input_rows_count, res_data, res_offset); + } else if (digest_length == 384) { + execute_base(data, offset, input_rows_count, res_data, res_offset); + } else if (digest_length == 512) { + execute_base(data, offset, input_rows_count, res_data, res_offset); + } else { + return Status::InvalidArgument( + "sha2's digest length only support 224/256/384/512 but meet {}", digest_length); + } + + block.replace_by_position(result, std::move(res_col)); + return Status::OK(); + } + +private: + template + void execute_base(const ColumnString::Chars& data, const ColumnString::Offsets& offset, + int input_rows_count, ColumnString::Chars& res_data, + ColumnString::Offsets& res_offset) { + T digest; + for (size_t i = 0; i < input_rows_count; ++i) { + int size = offset[i] - offset[i - 1]; + digest.reset(&data[offset[i - 1]], size); + std::string_view ans = digest.digest(); + + StringOP::push_value_string(ans, i, res_data, res_offset); + } + } +}; + class FunctionExtractURLParameter : public IFunction { public: static constexpr auto name = "extract_url_parameter"; diff --git a/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql b/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql index d2c4d93ef05db5c..829e2925a637e1d 100644 --- a/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql +++ b/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql @@ -123,4 +123,11 @@ create table doris_test."AA/D" ( create table doris_test.aaad ( id number(5), name varchar2(20) -); \ No newline at end of file +); + +create table doris_test."student2" ( +id number(5), +name varchar2(20), +age number(2), +score number(3,1) +); diff --git a/docker/thirdparties/docker-compose/oracle/init/04-insert.sql b/docker/thirdparties/docker-compose/oracle/init/04-insert.sql index b0b511121730832..17f68f994a237a2 100644 --- a/docker/thirdparties/docker-compose/oracle/init/04-insert.sql +++ b/docker/thirdparties/docker-compose/oracle/init/04-insert.sql @@ -80,4 +80,10 @@ insert into doris_test.test_clob values (10010, 'liantong'); insert into doris_test."AA/D" values (1, 'alice', 20, 99.5); insert into doris_test.aaad values (1, 'alice'); + +insert into doris_test."student2" values (1, 'alice', 20, 99.5); +insert into doris_test."student2" values (2, 'bob', 21, 90.5); +insert into doris_test."student2" values (3, 'jerry', 23, 88.0); +insert into doris_test."student2" values (4, 'andy', 21, 93); + commit; diff --git a/docs/en/docs/admin-manual/config/fe-config.md b/docs/en/docs/admin-manual/config/fe-config.md index 8fc84e6ef41f33c..736404361935a28 100644 --- a/docs/en/docs/admin-manual/config/fe-config.md +++ b/docs/en/docs/admin-manual/config/fe-config.md @@ -437,14 +437,6 @@ Default:1024 Maximal number of connections per FE. -#### `max_connection_scheduler_threads_num` - -Default:4096 - -Maximal number of thread in connection-scheduler-pool. - -The current strategy is to apply for a separate thread for service when there is a request - #### `check_java_version` Default:true diff --git a/docs/en/docs/lakehouse/multi-catalog/jdbc.md b/docs/en/docs/lakehouse/multi-catalog/jdbc.md index dcab8e056bfcdad..812f456318b4757 100644 --- a/docs/en/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/en/docs/lakehouse/multi-catalog/jdbc.md @@ -56,7 +56,8 @@ PROPERTIES ("key"="value", ...) | `include_database_list` | No | "" | When only_specified_database=true,only synchronize the specified databases. split with ','. db name is case sensitive. | | `exclude_database_list` | No | "" | When only_specified_database=true,do not synchronize the specified databases. split with ','. db name is case sensitive. | -:::tip +### Driver path + `driver_url` can be specified in three ways: 1. File name. For example, `mysql-connector-java-5.1.47.jar`. Please place the Jar file package in `jdbc_drivers/` under the FE/BE deployment directory in advance so the system can locate the file. You can change the location of the file by modifying `jdbc_drivers_dir` in fe.conf and be.conf. @@ -64,14 +65,30 @@ PROPERTIES ("key"="value", ...) 2. Local absolute path. For example, `file:///path/to/mysql-connector-java-5.1.47.jar`. Please place the Jar file package in the specified paths of FE/BE node. 3. HTTP address. For example, `https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/jdbc_driver/mysql-connector-java-8.0.25.jar`. The system will download the Driver file from the HTTP address. This only supports HTTP services with no authentication requirements. -::: -:::tip - `only_specified_database`: - When the JDBC is connected, you can specify which database/schema to connect. For example, you can specify the DataBase in mysql `jdbc_url`; you can specify the CurrentSchema in PG `jdbc_url`. +### Lowercase table name synchronization + +When `lower_case_table_names` is set to `true`, Doris is able to query non-lowercase databases and tables by maintaining a mapping of lowercase names to actual names on the remote system + +**Notice:** + +1. In versions before Doris 2.0.3, it is only valid for Oracle database. When querying, all library names and table names will be converted to uppercase before querying Oracle, for example: - `include_database_list`: - It only takes effect when `only_specified_database=true`, specify the database that needs to be synchronized, separated by ',', and the db name is case-sensitive. +Oracle has the TEST table in the TEST space. When Doris creates the Catalog, set `lower_case_table_names` to `true`, then Doris can query the TEST table through `select * from oracle_catalog.test.test`, and Doris will automatically format test.test into TEST.TEST is sent to Oracle. It should be noted that this is the default behavior, which also means that lowercase table names in Oracle cannot be queried. + + For other databases, you still need to specify the real library name and table name when querying. + +2. In Doris 2.0.3 and later versions, it is valid for all databases. When querying, all library names and table names will be converted into real names and then queried. If you upgrade from an old version to 2.0. 3, `Refresh ` is required to take effect. + + However, if the database or table names differ only in case, such as `Doris` and `doris`, Doris cannot query them due to ambiguity. + +### Specify synchronization database: + +`only_specified_database`: +When the JDBC is connected, you can specify which database/schema to connect. For example, you can specify the DataBase in mysql `jdbc_url`; you can specify the CurrentSchema in PG `jdbc_url`. + +`include_database_list`: +It only takes effect when `only_specified_database=true`, specify the database that needs to be synchronized, separated by ',', and the db name is case-sensitive. `exclude_database_list`: It only takes effect when `only specified database=true`, specifies multiple databases that do not need to be synchronized, separated by ',', and the db name is case-sensitive. @@ -79,7 +96,6 @@ It only takes effect when `only specified database=true`, specifies multiple dat When `include_database_list` and `exclude_database_list` specify overlapping databases, `exclude_database_list` would take effect with higher privilege over `include_database_list`. If you connect the Oracle database when using this property, please use the version of the jar package above 8 or more (such as ojdbc8.jar). -::: ## Query diff --git a/docs/en/docs/sql-manual/sql-functions/encrypt-digest-functions/sha.md b/docs/en/docs/sql-manual/sql-functions/encrypt-digest-functions/sha.md new file mode 100644 index 000000000000000..8e9a4573c1c37cf --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/encrypt-digest-functions/sha.md @@ -0,0 +1,53 @@ +--- +{ +"title": "SHA", +"language": "en" +} +--- + + + +## SHA + +### description + +Use SHA1 to digest the message. + +#### Syntax + +`SHA(str)` or `SHA1(str)` + +#### Arguments + +- `str`: content to be encrypted + +### example + +```SQL +mysql> select sha("123"); ++------------------------------------------+ +| sha1('123') | ++------------------------------------------+ +| 40bd001563085fc35165329ea1ff5c5ecbdbbeef | ++------------------------------------------+ +1 row in set (0.13 sec) +``` + +### keywords + + SHA,SHA1 diff --git a/docs/en/docs/sql-manual/sql-functions/encrypt-digest-functions/sha2.md b/docs/en/docs/sql-manual/sql-functions/encrypt-digest-functions/sha2.md new file mode 100644 index 000000000000000..2f7f838b1ffba77 --- /dev/null +++ b/docs/en/docs/sql-manual/sql-functions/encrypt-digest-functions/sha2.md @@ -0,0 +1,70 @@ +--- +{ +"title": "SHA2", +"language": "en" +} +--- + + + +## SHA2 + +### description + +Use SHA2 to digest the message. + +#### Syntax + +`SHA2(str, digest_length)` + +#### Arguments + +- `str`: content to be encrypted +- `digest_length`: the length of the digest + +### example + +```SQL +mysql> select sha2('abc', 224); ++----------------------------------------------------------+ +| sha2('abc', 224) | ++----------------------------------------------------------+ +| 23097d223405d8228642a477bda255b32aadbce4bda0b3f7e36c9da7 | ++----------------------------------------------------------+ +1 row in set (0.13 sec) + +mysql> select sha2('abc', 384); ++--------------------------------------------------------------------------------------------------+ +| sha2('abc', 384) | ++--------------------------------------------------------------------------------------------------+ +| cb00753f45a35e8bb5a03d699ac65007272c32ab0eded1631a8b605a43ff5bed8086072ba1e7cc2358baeca134c825a7 | ++--------------------------------------------------------------------------------------------------+ +1 row in set (0.13 sec) + +mysql> select sha2(NULL, 512); ++-----------------+ +| sha2(NULL, 512) | ++-----------------+ +| NULL | ++-----------------+ +1 row in set (0.09 sec) +``` + +### keywords + + SHA2 diff --git a/docs/sidebars.json b/docs/sidebars.json index d44dc08e89976b4..5bb5d7db51df731 100644 --- a/docs/sidebars.json +++ b/docs/sidebars.json @@ -713,7 +713,9 @@ "sql-manual/sql-functions/encrypt-digest-functions/md5sum", "sql-manual/sql-functions/encrypt-digest-functions/sm4", "sql-manual/sql-functions/encrypt-digest-functions/sm3", - "sql-manual/sql-functions/encrypt-digest-functions/sm3sum" + "sql-manual/sql-functions/encrypt-digest-functions/sm3sum", + "sql-manual/sql-functions/encrypt-digest-functions/sha", + "sql-manual/sql-functions/encrypt-digest-functions/sha2" ] }, { diff --git a/docs/zh-CN/docs/admin-manual/config/fe-config.md b/docs/zh-CN/docs/admin-manual/config/fe-config.md index e4c02ef8b468f88..0793c398102b4e3 100644 --- a/docs/zh-CN/docs/admin-manual/config/fe-config.md +++ b/docs/zh-CN/docs/admin-manual/config/fe-config.md @@ -437,14 +437,6 @@ FE https 使能标志位,false 表示支持 http,true 表示同时支持 htt 每个 FE 的最大连接数 -#### `max_connection_scheduler_threads_num` - -默认值:4096 - -查询请求调度器中的最大线程数。 - -目前的策略是,有请求过来,就为其单独申请一个线程进行服务 - #### `check_java_version` 默认值:true diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md index 35672e9835338a0..493f02e52546bc6 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/jdbc.md @@ -56,7 +56,8 @@ PROPERTIES ("key"="value", ...) | `include_database_list` | 否 | "" | 当only_specified_database=true时,指定同步多个database,以','分隔。db名称是大小写敏感的。 | | `exclude_database_list` | 否 | "" | 当only_specified_database=true时,指定不需要同步的多个database,以','分割。db名称是大小写敏感的。 | -:::tip +### 驱动包路径 + `driver_url` 可以通过以下三种方式指定: 1. 文件名。如 `mysql-connector-java-5.1.47.jar`。需将 Jar 包预先存放在 FE 和 BE 部署目录的 `jdbc_drivers/` 目录下。系统会自动在这个目录下寻找。该目录的位置,也可以由 fe.conf 和 be.conf 中的 `jdbc_drivers_dir` 配置修改。 @@ -64,9 +65,25 @@ PROPERTIES ("key"="value", ...) 2. 本地绝对路径。如 `file:///path/to/mysql-connector-java-5.1.47.jar`。需将 Jar 包预先存放在所有 FE/BE 节点指定的路径下。 3. Http 地址。如:`https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/jdbc_driver/mysql-connector-java-8.0.25.jar`。系统会从这个 http 地址下载 Driver 文件。仅支持无认证的 http 服务。 -::: -:::tip +### 小写表名同步 + +当 `lower_case_table_names` 设置为 `true` 时,Doris 通过维护小写名称到远程系统中实际名称的映射,能够查询非小写的数据库和表 + +**注意:** + +1. 在 Doris 2.0.3 之前的版本,仅对 Oracle 数据库有效,在查询时,会将所有的库名和表名转换为大写,再去查询 Oracle,例如: + + Oracle 在 TEST 空间下有 TEST 表,Doris 创建 Catalog 时设置 `lower_case_table_names` 为 `true`,则 Doris 可以通过 `select * from oracle_catalog.test.test` 查询到 TEST 表,Doris 会自动将 test.test 格式化成 TEST.TEST 下发到 Oracle,需要注意的是这是个默认行为,也意味着不能查询 Oracle 中小写的表名。 + + 对于其他数据库,仍需要在查询时指定真实的库名和表名。 + +2. 在 Doris 2.0.3 及之后的版本,对所有的数据库都有效,在查询时,会将所有的库名和表名转换为真实的名称,再去查询,如果是从老版本升级到 2.0.3 ,需要 `Refresh ` 才能生效。 + + 但是,如果数据库或者表名只有大小写不同,例如 `Doris` 和 `doris`,则 Doris 由于歧义而无法查询它们。 + +### 指定同步数据库 + `only_specified_database`: 在jdbc连接时可以指定链接到哪个database/schema, 如:mysql中jdbc_url中可以指定database, pg的jdbc_url中可以指定currentSchema。 @@ -79,7 +96,6 @@ PROPERTIES ("key"="value", ...) 当 `include_database_list` 和 `exclude_database_list` 有重合的database配置时,`exclude_database_list`会优先生效。 如果使用该参数时连接oracle数据库,要求使用ojdbc8.jar以上版本jar包。 -::: ## 数据查询 diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/encrypt-digest-functions/sha.md b/docs/zh-CN/docs/sql-manual/sql-functions/encrypt-digest-functions/sha.md new file mode 100644 index 000000000000000..1d0f4d593245416 --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/encrypt-digest-functions/sha.md @@ -0,0 +1,54 @@ +--- +{ +"title": "SHA", +"language": "zh-CN" +} +--- + + + +## SHA + +### description + +使用SHA1算法对信息进行摘要处理。 + +#### Syntax + +`SHA(str)` 或 `SHA1(str)` + +#### Arguments + +- `str`: 待加密的内容 + +### example + +```SQL +mysql> select sha("123"); ++------------------------------------------+ +| sha1('123') | ++------------------------------------------+ +| 40bd001563085fc35165329ea1ff5c5ecbdbbeef | ++------------------------------------------+ +1 row in set (0.13 sec) +``` + +### keywords + + SHA,SHA1 + diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/encrypt-digest-functions/sha2.md b/docs/zh-CN/docs/sql-manual/sql-functions/encrypt-digest-functions/sha2.md new file mode 100644 index 000000000000000..f4a55f31724060f --- /dev/null +++ b/docs/zh-CN/docs/sql-manual/sql-functions/encrypt-digest-functions/sha2.md @@ -0,0 +1,70 @@ +--- +{ +"title": "SHA2", +"language": "zh-CN" +} +--- + + + +## SHA2 + +### description + +使用SHA2对信息进行摘要处理。 + +#### Syntax + +`SHA2(str, digest_length)` + +#### Arguments + +- `str`: 待加密的内容 +- `digest_length`: 摘要长度 + +### example + +```SQL +mysql> select sha2('abc', 224); ++----------------------------------------------------------+ +| sha2('abc', 224) | ++----------------------------------------------------------+ +| 23097d223405d8228642a477bda255b32aadbce4bda0b3f7e36c9da7 | ++----------------------------------------------------------+ +1 row in set (0.13 sec) + +mysql> select sha2('abc', 384); ++--------------------------------------------------------------------------------------------------+ +| sha2('abc', 384) | ++--------------------------------------------------------------------------------------------------+ +| cb00753f45a35e8bb5a03d699ac65007272c32ab0eded1631a8b605a43ff5bed8086072ba1e7cc2358baeca134c825a7 | ++--------------------------------------------------------------------------------------------------+ +1 row in set (0.13 sec) + +mysql> select sha2(NULL, 512); ++-----------------+ +| sha2(NULL, 512) | ++-----------------+ +| NULL | ++-----------------+ +1 row in set (0.09 sec) +``` + +### keywords + + SHA2 diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/ArrayType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/ArrayType.java index d3bd642a6d07a30..477d278bf8a24bd 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/ArrayType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/ArrayType.java @@ -203,7 +203,7 @@ public boolean supportSubType(Type subType) { @Override public String toString() { - return toSql(0).toUpperCase(); + return String.format("ARRAY<%s>", itemType.toString()).toUpperCase(); } @Override diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/MapType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/MapType.java index 82277706360b0f9..e9efc83a8fcbde1 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/MapType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/MapType.java @@ -174,7 +174,8 @@ public Type specializeTemplateType(Type specificType, Map speciali @Override public String toString() { - return toSql(0).toUpperCase(); + return String.format("MAP<%s,%s>", + keyType.toString(), valueType.toString()); } @Override diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/StructField.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/StructField.java index 488e477a07eb95d..5a2b3f221ea2449 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/StructField.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/StructField.java @@ -158,4 +158,16 @@ public boolean equals(Object other) { return otherStructField.name.equals(name) && otherStructField.type.equals(type) && otherStructField.containsNull == containsNull; } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(name); + if (type != null) { + sb.append(":").append(type); + } + if (StringUtils.isNotBlank(comment)) { + sb.append(String.format(" COMMENT '%s'", comment)); + } + return sb.toString(); + } } diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/StructType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/StructType.java index bd1efec99b5f7a8..0a058ae5406f859 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/StructType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/StructType.java @@ -303,7 +303,11 @@ public void toThrift(TTypeDesc container) { @Override public String toString() { - return toSql(0); + ArrayList fieldsSql = Lists.newArrayList(); + for (StructField f : fields) { + fieldsSql.add(f.toString()); + } + return String.format("STRUCT<%s>", Joiner.on(",").join(fieldsSql)); } @Override 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 a2aa07ba24b7fef..6e9eff590514314 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 @@ -702,10 +702,6 @@ public class Config extends ConfigBase { "Maximal number of connections of MySQL server per FE."}) public static int qe_max_connection = 1024; - @ConfField(description = {"MySQL 连接调度线程池的最大线程数。", - "Maximal number of thread in MySQL connection-scheduler-pool."}) - public static int max_connection_scheduler_threads_num = 4096; - @ConfField(mutable = true, description = {"Colocate join 每个 instance 的内存 penalty 系数。" + "计算方式:`exec_mem_limit / min (query_colocate_join_memory_limit_penalty_factor, instance_num)`", "Colocate join PlanFragment instance memory limit penalty factor.", diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java index 310c6a6d3bbc39e..a161685dde91ca5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java @@ -1704,6 +1704,19 @@ && collectChildReturnTypes()[0].isDecimalV3()) { } } + if (fn.getFunctionName().getFunction().equals("sha2")) { + if ((children.size() != 2) || (getChild(1).isConstant() == false) + || !(getChild(1) instanceof IntLiteral)) { + throw new AnalysisException( + fnName.getFunction() + " needs two params, and the second is must be a integer constant: " + + this.toSql()); + } + final Integer constParam = (int) ((IntLiteral) getChild(1)).getValue(); + if (!Lists.newArrayList(224, 256, 384, 512).contains(constParam)) { + throw new AnalysisException("sha2 functions only support digest length of 224/256/384/512"); + } + } + if (isAggregateFunction()) { final String functionName = fnName.getFunction(); // subexprs must not contain aggregates diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/Snapshot.java b/fe/fe-core/src/main/java/org/apache/doris/backup/Snapshot.java index b26cb2e1e7b337c..e31309b19a59eed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/Snapshot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/Snapshot.java @@ -31,9 +31,6 @@ public class Snapshot { @SerializedName(value = "jobInfo") private byte[] jobInfo = null; - @SerializedName(value = "createTime") - private String createTime = null; - public Snapshot() { } @@ -63,7 +60,6 @@ public String toString() { + "label='" + label + '\'' + ", meta=" + meta + ", jobInfo=" + jobInfo - + ", createTime='" + createTime + '\'' + '}'; } } 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 0666d87b7176d5c..37f534d770bfaa4 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 @@ -294,6 +294,8 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Sha1; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Sha2; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sign; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sin; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sleep; @@ -690,6 +692,8 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(SecondTimestamp.class, "second_timestamp"), scalar(MilliSecondTimestamp.class, "millisecond_timestamp"), scalar(MicroSecondTimestamp.class, "microsecond_timestamp"), + scalar(Sha1.class, "sha1", "sha"), + scalar(Sha2.class, "sha2"), scalar(Sign.class, "sign"), scalar(Sin.class, "sin"), scalar(Sleep.class, "sleep"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java index 7d7e6d545a12aaf..795f00e7bfed9ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java @@ -48,6 +48,8 @@ public class JdbcTable extends Table { private static final Logger LOG = LogManager.getLogger(JdbcTable.class); private static final String TABLE = "table"; + private static final String REAL_DATABASE = "real_database"; + private static final String REAL_TABLE = "real_table"; private static final String RESOURCE = "resource"; private static final String TABLE_TYPE = "table_type"; private static final String URL = "jdbc_url"; @@ -59,6 +61,11 @@ public class JdbcTable extends Table { private static Map TABLE_TYPE_MAP; private String resourceName; private String externalTableName; + + // real name only for jdbc catalog + private String realDatabaseName; + private String realTableName; + private String jdbcTypeName; private String jdbcUrl; @@ -100,8 +107,7 @@ public JdbcTable(long id, String name, List schema, TableType type) { public String getInsertSql(List insertCols) { StringBuilder sb = new StringBuilder("INSERT INTO "); - - sb.append(databaseProperName(TABLE_TYPE_MAP.get(getTableTypeName()), getExternalTableName())); + sb.append(getProperRealFullTableName(TABLE_TYPE_MAP.get(getTableTypeName()))); sb.append("("); List transformedInsertCols = insertCols.stream() .map(col -> databaseProperName(TABLE_TYPE_MAP.get(getTableTypeName()), col)) @@ -192,6 +198,8 @@ public void write(DataOutput out) throws IOException { serializeMap.put(DRIVER_CLASS, driverClass); serializeMap.put(DRIVER_URL, driverUrl); serializeMap.put(CHECK_SUM, checkSum); + serializeMap.put(REAL_DATABASE, realDatabaseName); + serializeMap.put(REAL_TABLE, realTableName); int size = (int) serializeMap.values().stream().filter(v -> { return v != null; @@ -226,6 +234,8 @@ public void readFields(DataInput in) throws IOException { driverClass = serializeMap.get(DRIVER_CLASS); driverUrl = serializeMap.get(DRIVER_URL); checkSum = serializeMap.get(CHECK_SUM); + realDatabaseName = serializeMap.get(REAL_DATABASE); + realTableName = serializeMap.get(REAL_TABLE); } public String getResourceName() { @@ -236,6 +246,23 @@ public String getJdbcTable() { return externalTableName; } + public String getRealDatabaseName() { + return realDatabaseName; + } + + public String getRealTableName() { + return realTableName; + } + + public String getProperRealFullTableName(TOdbcTableType tableType) { + if (realDatabaseName == null || realTableName == null) { + return databaseProperName(tableType, externalTableName); + } else { + return properNameWithRealName(tableType, realDatabaseName) + "." + properNameWithRealName(tableType, + realTableName); + } + } + public String getTableTypeName() { return jdbcTypeName; } @@ -342,7 +369,7 @@ private void validate(Map properties) throws DdlException { * @return The formatted name. */ public static String formatName(String name, String wrapStart, String wrapEnd, boolean toUpperCase, - boolean toLowerCase) { + boolean toLowerCase) { int index = name.indexOf("."); if (index == -1) { // No dot in the name String newName = toUpperCase ? name.toUpperCase() : name; @@ -393,4 +420,28 @@ public static String databaseProperName(TOdbcTableType tableType, String name) { return name; } } + + public static String properNameWithRealName(TOdbcTableType tableType, String name) { + switch (tableType) { + case MYSQL: + case OCEANBASE: + return formatNameWithRealName(name, "`", "`"); + case SQLSERVER: + return formatNameWithRealName(name, "[", "]"); + case POSTGRESQL: + case CLICKHOUSE: + case TRINO: + case PRESTO: + case OCEANBASE_ORACLE: + case ORACLE: + case SAP_HANA: + return formatNameWithRealName(name, "\"", "\""); + default: + return name; + } + } + + public static String formatNameWithRealName(String name, String wrapStart, String wrapEnd) { + return wrapStart + name + wrapEnd; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java index 676817654f6be67..68c391a2a29fd23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java @@ -88,6 +88,9 @@ private JdbcTable toJdbcTable() { String fullDbName = this.dbName + "." + this.name; JdbcTable jdbcTable = new JdbcTable(this.id, fullDbName, schema, TableType.JDBC_EXTERNAL_TABLE); jdbcTable.setExternalTableName(fullDbName); + jdbcTable.setRealDatabaseName(((JdbcExternalCatalog) catalog).getJdbcClient().getRealDatabaseName(this.dbName)); + jdbcTable.setRealTableName( + ((JdbcExternalCatalog) catalog).getJdbcClient().getRealTableName(this.dbName, this.name)); jdbcTable.setJdbcTypeName(jdbcCatalog.getDatabaseTypeName()); jdbcTable.setJdbcUrl(jdbcCatalog.getJdbcUrl()); jdbcTable.setJdbcUser(jdbcCatalog.getJdbcUser()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index 8ffc5df3cabcde7..97052caccd6e2f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -26,7 +26,6 @@ import com.alibaba.druid.pool.DruidDataSource; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import lombok.Data; import lombok.Getter; import org.apache.logging.log4j.LogManager; @@ -44,6 +43,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; @Getter @@ -64,9 +65,12 @@ public abstract class JdbcClient { protected Map includeDatabaseMap; protected Map excludeDatabaseMap; // only used when isLowerCaseTableNames = true. - protected Map lowerTableToRealTable = Maps.newHashMap(); + protected final ConcurrentHashMap lowerDBToRealDB = new ConcurrentHashMap<>(); // only used when isLowerCaseTableNames = true. - protected Map lowerDBToRealDB = Maps.newHashMap(); + protected final ConcurrentHashMap lowerTableToRealTable = new ConcurrentHashMap<>(); + + private final AtomicBoolean dbNamesLoaded = new AtomicBoolean(false); + private final AtomicBoolean tableNamesLoaded = new AtomicBoolean(false); public static JdbcClient createJdbcClient(JdbcClientConfig jdbcClientConfig) { String dbType = parseDbType(jdbcClientConfig.getJdbcUrl()); @@ -237,11 +241,7 @@ public List getTablesNameList(String dbName) { List tablesName = Lists.newArrayList(); String[] tableTypes = getTableTypes(); if (isLowerCaseTableNames) { - currentDbName = lowerDBToRealDB.get(dbName); - if (currentDbName == null) { - getDatabaseNameList(); - currentDbName = lowerDBToRealDB.get(dbName); - } + currentDbName = getRealDatabaseName(dbName); } String finalDbName = currentDbName; processTable(finalDbName, null, tableTypes, (rs) -> { @@ -266,16 +266,8 @@ public boolean isTableExist(String dbName, String tableName) { String currentTableName = tableName; final boolean[] isExist = {false}; if (isLowerCaseTableNames) { - currentDbName = lowerDBToRealDB.get(dbName); - currentTableName = lowerTableToRealTable.get(tableName); - if (currentDbName == null) { - getDatabaseNameList(); - currentDbName = lowerDBToRealDB.get(dbName); - } - if (currentTableName == null) { - getTablesNameList(dbName); - currentTableName = lowerTableToRealTable.get(tableName); - } + currentDbName = getRealDatabaseName(dbName); + currentTableName = getRealTableName(dbName, tableName); } String[] tableTypes = getTableTypes(); String finalTableName = currentTableName; @@ -305,16 +297,8 @@ public List getJdbcColumnsInfo(String dbName, String tableName) String currentDbName = dbName; String currentTableName = tableName; if (isLowerCaseTableNames) { - currentDbName = lowerDBToRealDB.get(dbName); - currentTableName = lowerTableToRealTable.get(tableName); - if (currentDbName == null) { - getDatabaseNameList(); - currentDbName = lowerDBToRealDB.get(dbName); - } - if (currentTableName == null) { - getTablesNameList(dbName); - currentTableName = lowerTableToRealTable.get(tableName); - } + currentDbName = getRealDatabaseName(dbName); + currentTableName = getRealTableName(dbName, tableName); } String finalDbName = currentDbName; String finalTableName = currentTableName; @@ -367,6 +351,42 @@ public List getColumnsFromJdbc(String dbName, String tableName) { return dorisTableSchema; } + public String getRealDatabaseName(String dbname) { + if (!isLowerCaseTableNames) { + return dbname; + } + + if (lowerDBToRealDB.isEmpty() || !lowerDBToRealDB.containsKey(dbname)) { + loadDatabaseNamesIfNeeded(); + } + + return lowerDBToRealDB.get(dbname); + } + + public String getRealTableName(String dbName, String tableName) { + if (!isLowerCaseTableNames) { + return tableName; + } + + if (lowerTableToRealTable.isEmpty() || !lowerTableToRealTable.containsKey(tableName)) { + loadTableNamesIfNeeded(dbName); + } + + return lowerTableToRealTable.get(tableName); + } + + private void loadDatabaseNamesIfNeeded() { + if (dbNamesLoaded.compareAndSet(false, true)) { + getDatabaseNameList(); + } + } + + private void loadTableNamesIfNeeded(String dbName) { + if (tableNamesLoaded.compareAndSet(false, true)) { + getTablesNameList(dbName); + } + } + // protected methods,for subclass to override protected String getCatalogName(Connection conn) throws SQLException { return null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcOracleClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcOracleClient.java index 410e10df20c5463..d0a9f2c3de7ab3f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcOracleClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcOracleClient.java @@ -94,16 +94,8 @@ public List getJdbcColumnsInfo(String dbName, String tableName) String currentDbName = dbName; String currentTableName = tableName; if (isLowerCaseTableNames) { - currentDbName = lowerDBToRealDB.get(dbName); - currentTableName = lowerTableToRealTable.get(tableName); - if (currentDbName == null) { - getDatabaseNameList(); - currentDbName = lowerDBToRealDB.get(dbName); - } - if (currentTableName == null) { - getTablesNameList(dbName); - currentTableName = lowerTableToRealTable.get(tableName); - } + currentDbName = getRealDatabaseName(dbName); + currentTableName = getRealTableName(dbName, tableName); } String finalDbName = currentDbName; String finalTableName = currentTableName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java index 3945861d5446416..9d51152f1fb996c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java @@ -79,6 +79,9 @@ public class GroupExpression { private final ObjectId id = StatementScopeIdGenerator.newObjectId(); + /** + * Just for UT. + */ public GroupExpression(Plan plan) { this(plan, Lists.newArrayList()); } @@ -302,8 +305,7 @@ public boolean equals(Object o) { return false; } GroupExpression that = (GroupExpression) o; - return children.equals(that.children) && plan.equals(that.plan) - && plan.getLogicalProperties().equals(that.plan.getLogicalProperties()); + return children.equals(that.children) && plan.equals(that.plan); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java index ade8eb8cc6dcce0..804ebcda84eb4d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupExpressionMatching.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -68,18 +69,19 @@ public GroupExpressionIterator(Pattern pattern, GroupExpression groupExpre } int childrenGroupArity = groupExpression.arity(); + int patternArity = pattern.arity(); if (!(pattern instanceof SubTreePattern)) { // (logicalFilter(), multi()) match (logicalFilter()), // but (logicalFilter(), logicalFilter(), multi()) not match (logicalFilter()) - boolean extraMulti = pattern.arity() == childrenGroupArity + 1 + boolean extraMulti = patternArity == childrenGroupArity + 1 && (pattern.hasMultiChild() || pattern.hasMultiGroupChild()); - if (pattern.arity() > childrenGroupArity && !extraMulti) { + if (patternArity > childrenGroupArity && !extraMulti) { return; } // (multi()) match (logicalFilter(), logicalFilter()), // but (logicalFilter()) not match (logicalFilter(), logicalFilter()) - if (!pattern.isAny() && pattern.arity() < childrenGroupArity + if (!pattern.isAny() && patternArity < childrenGroupArity && !pattern.hasMultiChild() && !pattern.hasMultiGroupChild()) { return; } @@ -92,7 +94,7 @@ public GroupExpressionIterator(Pattern pattern, GroupExpression groupExpre // getPlan return the plan with GroupPlan as children Plan root = groupExpression.getPlan(); - if (pattern.arity() == 0 && !(pattern instanceof SubTreePattern)) { + if (patternArity == 0 && !(pattern instanceof SubTreePattern)) { if (pattern.matchPredicates(root)) { // if no children pattern, we treat all children as GROUP. e.g. Pattern.ANY. // leaf plan will enter this branch too, e.g. logicalRelation(). @@ -118,7 +120,7 @@ public GroupExpressionIterator(Pattern pattern, GroupExpression groupExpre childrenPlans.add(childrenPlan); } assembleAllCombinationPlanTree(root, pattern, groupExpression, childrenPlans); - } else if (pattern.arity() == 1 && (pattern.hasMultiChild() || pattern.hasMultiGroupChild())) { + } else if (patternArity == 1 && (pattern.hasMultiChild() || pattern.hasMultiGroupChild())) { // leaf group with multi child pattern // e.g. logicalPlan(multi()) match LogicalOlapScan, because LogicalOlapScan is LogicalPlan // and multi() pattern indicate zero or more children() @@ -148,9 +150,9 @@ private List matchingChildGroup(Pattern parentPattern, } } - ImmutableList.Builder matchingChildren = ImmutableList.builder(); + List matchingChildren = new ArrayList<>(); new GroupMatching(childPattern, childGroup).forEach(matchingChildren::add); - return matchingChildren.build(); + return matchingChildren; } private void assembleAllCombinationPlanTree(Plan root, Pattern rootPattern, @@ -158,6 +160,7 @@ private void assembleAllCombinationPlanTree(Plan root, Pattern rootPattern List> childrenPlans) { int[] childrenPlanIndex = new int[childrenPlans.size()]; int offset = 0; + LogicalProperties logicalProperties = groupExpression.getOwnerGroup().getLogicalProperties(); // assemble all combination of plan tree by current root plan and children plan while (offset < childrenPlans.size()) { @@ -168,7 +171,6 @@ private void assembleAllCombinationPlanTree(Plan root, Pattern rootPattern } List children = childrenBuilder.build(); - LogicalProperties logicalProperties = groupExpression.getOwnerGroup().getLogicalProperties(); // assemble children: replace GroupPlan to real plan, // withChildren will erase groupExpression, so we must // withGroupExpression too. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupMatching.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupMatching.java index 8429902e593a24e..a76a07d7fd11575 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupMatching.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/GroupMatching.java @@ -23,8 +23,8 @@ import org.apache.doris.nereids.trees.plans.Plan; import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -42,7 +42,7 @@ public GroupMatching(Pattern pattern, Group group) { this.group = Objects.requireNonNull(group); } - public Iterator iterator() { + public final Iterator iterator() { return new GroupIterator(pattern, group); } @@ -60,7 +60,7 @@ public static class GroupIterator implements Iterator { * @param group group to be matched */ public GroupIterator(Pattern pattern, Group group) { - this.iterator = Lists.newArrayList(); + this.iterator = new ArrayList<>(); if (pattern.isGroup() || pattern.isMultiGroup()) { GroupPlan groupPlan = new GroupPlan(group); @@ -86,12 +86,12 @@ public GroupIterator(Pattern pattern, Group group) { } @Override - public boolean hasNext() { + public final boolean hasNext() { return iteratorIndex < iterator.size(); } @Override - public Plan next() { + public final Plan next() { if (!hasNext()) { throw new NoSuchElementException(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index c315f29981d15ef..2a2a529ff918734 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -143,7 +143,7 @@ protected boolean condition(Rule rule, Plan plan) { boundProjections = boundProjections.stream() .map(expr -> bindFunction(expr, ctx.root, ctx.cascadesContext)) .collect(ImmutableList.toImmutableList()); - return new LogicalProject<>(boundProjections, project.isDistinct(), project.child()); + return project.withProjects(boundProjections); }) ), RuleType.BINDING_FILTER_SLOT.build( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java index 71818966696958e..1c1a93e39eb10d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PredicatePropagation.java @@ -127,7 +127,7 @@ private Expression replaceSlot(Expression sourcePredicate, DataType originDataTy private boolean canEquivalentInfer(Expression predicate) { return predicate instanceof EqualTo && predicate.children().stream().allMatch(e -> - (e instanceof SlotReference) || (e instanceof Cast && e.child(0).isSlot())) + (e instanceof SlotReference) || (e instanceof Cast && e.child(0) instanceof SlotReference)) && predicate.child(0).getDataType().equals(predicate.child(1).getDataType()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java index 08bd198e4429e7c..2e18dd4bacdff58 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java @@ -17,14 +17,12 @@ package org.apache.doris.nereids.trees; -import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.ObjectId; import com.google.common.collect.ImmutableList; import java.util.List; -import java.util.Optional; /** * Abstract class for plan node in Nereids, include plan node and expression. @@ -39,11 +37,11 @@ public abstract class AbstractTreeNode> // TODO: Maybe we should use a GroupPlan to avoid TreeNode hold the GroupExpression. // https://github.com/apache/doris/pull/9807#discussion_r884829067 - public AbstractTreeNode(NODE_TYPE... children) { - this(Optional.empty(), ImmutableList.copyOf(children)); + protected AbstractTreeNode(NODE_TYPE... children) { + this.children = ImmutableList.copyOf(children); } - public AbstractTreeNode(Optional groupExpression, List children) { + protected AbstractTreeNode(List children) { this.children = ImmutableList.copyOf(children); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java index c5675ba63d729b0..07c0831a0f0efe6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java @@ -36,14 +36,12 @@ import org.apache.doris.nereids.types.coercion.AnyDataType; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; import java.util.Arrays; import java.util.List; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -69,7 +67,7 @@ protected Expression(Expression... children) { } protected Expression(List children) { - super(Optional.empty(), children); + super(children); depth = children.stream() .mapToInt(e -> e.depth) .max().orElse(0) + 1; @@ -209,10 +207,6 @@ public Expression withChildren(List children) { throw new RuntimeException(); } - public final Expression withChildren(Expression... children) { - return withChildren(ImmutableList.copyOf(children)); - } - /** * Whether the expression is a constant. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sha1.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sha1.java new file mode 100644 index 000000000000000..b405e5440866b0c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sha1.java @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +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.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 'sha1'. This class is not generated by GenerateFunction. + */ +public class Sha1 extends ScalarFunction + implements ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(StringType.INSTANCE)); + + /** + * constructor with 1 arguments. + */ + public Sha1(Expression arg0) { + super("sha1", arg0); + } + + /** + * withChildren. + */ + @Override + public Sha1 withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new Sha1(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitSha1(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sha2.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sha2.java new file mode 100644 index 000000000000000..2b37e66943d5e5f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Sha2.java @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.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.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.IntegerType; +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 'sha2'. This class is not generated by GenerateFunction. + */ +public class Sha2 extends ScalarFunction + implements ExplicitlyCastableSignature, PropagateNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(VarcharType.SYSTEM_DEFAULT, IntegerType.INSTANCE), + FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args(StringType.INSTANCE, IntegerType.INSTANCE)); + + /** + * constructor with 2 arguments. + */ + public Sha2(Expression arg0, Expression arg1) { + super("sha2", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public Sha2 withChildren(List children) { + Preconditions.checkArgument(children.size() == 2); + return new Sha2(children.get(0), children.get(1)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitSha2(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 c15f03f2bfc71cc..b0dde6aff21827b 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 @@ -290,6 +290,8 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsAdd; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsDiff; import org.apache.doris.nereids.trees.expressions.functions.scalar.SecondsSub; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Sha1; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Sha2; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sign; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sin; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sleep; @@ -1434,6 +1436,14 @@ default R visitSecondsDiff(SecondsDiff secondsDiff, C context) { return visitScalarFunction(secondsDiff, context); } + default R visitSha1(Sha1 sha1, C context) { + return visitScalarFunction(sha1, context); + } + + default R visitSha2(Sha2 sha2, C context) { + return visitScalarFunction(sha2, context); + } + default R visitMilliSecondsDiff(MilliSecondsDiff milliSecondsDiff, C context) { return visitScalarFunction(milliSecondsDiff, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java index 86ae62e1228cd61..38a209ff55f49b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java @@ -31,7 +31,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import com.google.common.collect.ImmutableList; import org.json.JSONArray; import org.json.JSONObject; @@ -60,7 +59,7 @@ public abstract class AbstractPlan extends AbstractTreeNode implements Pla private MutableState mutableState = EmptyMutableState.INSTANCE; protected AbstractPlan(PlanType type, List children) { - this(type, Optional.empty(), Optional.empty(), null, ImmutableList.copyOf(children)); + this(type, Optional.empty(), Optional.empty(), null, children); } /** @@ -69,7 +68,7 @@ protected AbstractPlan(PlanType type, List children) { protected AbstractPlan(PlanType type, Optional groupExpression, Optional optLogicalProperties, @Nullable Statistics statistics, List children) { - super(groupExpression, children); + super(children); this.type = Objects.requireNonNull(type, "type can not be null"); this.groupExpression = Objects.requireNonNull(groupExpression, "groupExpression can not be null"); Objects.requireNonNull(optLogicalProperties, "logicalProperties can not be null"); @@ -114,7 +113,7 @@ public String treeString() { public JSONObject toJson() { JSONObject json = new JSONObject(); json.put("PlanType", getType().toString()); - if (this.children().size() == 0) { + if (this.children().isEmpty()) { return json; } JSONArray childrenJson = new JSONArray(); 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 bafb1d429b68e21..b5459acf35b2f18 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 @@ -123,17 +123,15 @@ public LogicalJoin( Optional.empty(), Optional.empty(), children); } - /** - * Just use in withXXX method. - */ private LogicalJoin(JoinType joinType, List hashJoinConjuncts, List otherJoinConjuncts, JoinHint hint, Optional markJoinSlotReference, Optional groupExpression, Optional logicalProperties, List children, JoinReorderContext joinReorderContext) { + // Just use in withXXX method. Don't need check/copyOf() super(PlanType.LOGICAL_JOIN, groupExpression, logicalProperties, children); this.joinType = Objects.requireNonNull(joinType, "joinType can not be null"); - this.hashJoinConjuncts = ImmutableList.copyOf(hashJoinConjuncts); - this.otherJoinConjuncts = ImmutableList.copyOf(otherJoinConjuncts); + this.hashJoinConjuncts = hashJoinConjuncts; + this.otherJoinConjuncts = otherJoinConjuncts; this.hint = Objects.requireNonNull(hint, "hint can not be null"); this.joinReorderContext.copyFrom(joinReorderContext); this.markJoinSlotReference = markJoinSlotReference; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalLeaf.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalLeaf.java index f874ae8c4fa1b2f..a87f408bf31e7c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalLeaf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalLeaf.java @@ -23,6 +23,8 @@ import org.apache.doris.nereids.trees.plans.LeafPlan; import org.apache.doris.nereids.trees.plans.PlanType; +import com.google.common.collect.ImmutableList; + import java.util.List; import java.util.Optional; @@ -33,7 +35,7 @@ public abstract class LogicalLeaf extends AbstractLogicalPlan implements LeafPla public LogicalLeaf(PlanType nodeType, Optional groupExpression, Optional logicalProperties) { - super(nodeType, groupExpression, logicalProperties); + super(nodeType, groupExpression, logicalProperties, ImmutableList.of()); } public abstract List computeOutput(); 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 283d09767301072..46b3f3d84a19731 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 @@ -51,28 +51,22 @@ public class LogicalProject extends LogicalUnary projects, CHILD_TYPE child) { - this(projects, ImmutableList.of(), false, child); + this(projects, ImmutableList.of(), false, ImmutableList.of(child)); } - /** - * only for test. - */ - public LogicalProject(List projects, List excepts, CHILD_TYPE child) { - this(projects, excepts, false, child); - } - - public LogicalProject(List projects, boolean isDistinct, CHILD_TYPE child) { - this(projects, ImmutableList.of(), isDistinct, child); + public LogicalProject(List projects, List excepts, + boolean isDistinct, List child) { + this(projects, excepts, isDistinct, Optional.empty(), Optional.empty(), child); } public LogicalProject(List projects, List excepts, - boolean isDistinct, CHILD_TYPE child) { - this(projects, excepts, isDistinct, Optional.empty(), Optional.empty(), child); + boolean isDistinct, Plan child) { + this(projects, excepts, isDistinct, Optional.empty(), Optional.empty(), ImmutableList.of(child)); } private LogicalProject(List projects, List excepts, boolean isDistinct, Optional groupExpression, Optional logicalProperties, - CHILD_TYPE child) { + List child) { super(PlanType.LOGICAL_PROJECT, groupExpression, logicalProperties, child); Preconditions.checkArgument(projects != null, "projects can not be null"); // only ColumnPrune rule may produce empty projects, this happens in rewrite phase @@ -80,7 +74,7 @@ private LogicalProject(List projects, List exc Preconditions.checkArgument(!projects.isEmpty() || !(child instanceof Unbound), "projects can not be empty when child plan is unbound"); this.projects = projects.isEmpty() - ? ImmutableList.of(ExpressionUtils.selectMinimumColumn(child.getOutput())) + ? ImmutableList.of(ExpressionUtils.selectMinimumColumn(child.get(0).getOutput())) : projects; this.excepts = ImmutableList.copyOf(excepts); this.isDistinct = isDistinct; @@ -157,13 +151,13 @@ public int hashCode() { @Override public LogicalProject withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalProject<>(projects, excepts, isDistinct, children.get(0)); + return new LogicalProject<>(projects, excepts, isDistinct, ImmutableList.copyOf(children)); } @Override public LogicalProject withGroupExpression(Optional groupExpression) { return new LogicalProject<>(projects, excepts, isDistinct, - groupExpression, Optional.of(getLogicalProperties()), child()); + groupExpression, Optional.of(getLogicalProperties()), children); } @Override @@ -171,15 +165,15 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); return new LogicalProject<>(projects, excepts, isDistinct, - groupExpression, logicalProperties, children.get(0)); + groupExpression, logicalProperties, children); } public LogicalProject withProjects(List projects) { - return new LogicalProject<>(projects, excepts, isDistinct, child()); + return new LogicalProject<>(projects, excepts, isDistinct, children); } public LogicalProject withProjectsAndChild(List projects, Plan child) { - return new LogicalProject<>(projects, excepts, isDistinct, child); + return new LogicalProject<>(projects, excepts, isDistinct, ImmutableList.of(child)); } public boolean isDistinct() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnary.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnary.java index 70b5ec12bc96363..f067f055c39e8e7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnary.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnary.java @@ -36,14 +36,19 @@ public abstract class LogicalUnary extends AbstractLogicalPlan implements UnaryPlan { - public LogicalUnary(PlanType type, CHILD_TYPE child) { + protected LogicalUnary(PlanType type, CHILD_TYPE child) { super(type, ImmutableList.of(child)); } - public LogicalUnary(PlanType type, Optional groupExpression, + protected LogicalUnary(PlanType type, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { super(type, groupExpression, logicalProperties, child); } + protected LogicalUnary(PlanType type, Optional groupExpression, + Optional logicalProperties, List child) { + super(type, groupExpression, logicalProperties, child); + } + public abstract List computeOutput(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java index 1fefd749d424a2b..614b71484ce95ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcScanNode.java @@ -80,7 +80,7 @@ public JdbcScanNode(PlanNodeId id, TupleDescriptor desc, boolean isJdbcExternalT tbl = (JdbcTable) (desc.getTable()); } jdbcType = tbl.getJdbcTableType(); - tableName = JdbcTable.databaseProperName(jdbcType, tbl.getJdbcTable()); + tableName = tbl.getProperRealFullTableName(jdbcType); } @Override @@ -130,7 +130,7 @@ private void createJdbcFilters() { for (SlotRef slotRef : slotRefs) { SlotRef slotRef1 = (SlotRef) slotRef.clone(); slotRef1.setTblName(null); - slotRef1.setLabel(JdbcTable.databaseProperName(jdbcType, slotRef1.getColumnName())); + slotRef1.setLabel(JdbcTable.properNameWithRealName(jdbcType, slotRef1.getColumnName())); sMap.put(slotRef, slotRef1); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcTableSink.java index 263860f1e78bea7..6a767ff18569bc5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/jdbc/JdbcTableSink.java @@ -53,7 +53,7 @@ public class JdbcTableSink extends DataSink { public JdbcTableSink(JdbcTable jdbcTable, List insertCols) { resourceName = jdbcTable.getResourceName(); jdbcType = jdbcTable.getJdbcTableType(); - externalTableName = JdbcTable.databaseProperName(jdbcType, jdbcTable.getExternalTableName()); + externalTableName = jdbcTable.getProperRealFullTableName(jdbcType); useTransaction = ConnectContext.get().getSessionVariable().isEnableOdbcTransaction(); jdbcUrl = jdbcTable.getJdbcUrl(); jdbcUser = jdbcTable.getJdbcUser(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java index 8ce1615bb294fd5..a89324de390ef6c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectScheduler.java @@ -18,7 +18,6 @@ package org.apache.doris.qe; import org.apache.doris.catalog.Env; -import org.apache.doris.common.Config; import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -32,7 +31,6 @@ import java.util.List; import java.util.Map; import java.util.TimerTask; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -47,8 +45,6 @@ public class ConnectScheduler { private final AtomicInteger nextConnectionId; private final Map connectionMap = Maps.newConcurrentMap(); private final Map connByUser = Maps.newConcurrentMap(); - private final ExecutorService executor = ThreadPoolManager.newDaemonCacheThreadPool( - Config.max_connection_scheduler_threads_num, "connect-scheduler-pool", true); // valid trace id -> query id private final Map traceId2QueryId = Maps.newConcurrentMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java b/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java index 45bddda1e42b0bb..f6eaaea25445373 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/QeProcessorImpl.java @@ -209,7 +209,7 @@ public TReportExecStatusResult reportExecStatus(TReportExecStatusParams params, } else { result.setStatus(new TStatus(TStatusCode.RUNTIME_ERROR)); } - LOG.info("ReportExecStatus() runtime error, query {} with type {} does not exist", + LOG.warn("ReportExecStatus() runtime error, query {} with type {} does not exist", DebugUtil.printId(params.query_id), params.query_type); return result; } 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 4238e012c5ef2cd..3eaaeaa14ad3c32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -18,6 +18,7 @@ package org.apache.doris.service; import org.apache.doris.alter.SchemaChangeHandler; +import org.apache.doris.analysis.AbstractBackupTableRefClause; import org.apache.doris.analysis.AddColumnsClause; import org.apache.doris.analysis.AddPartitionClause; import org.apache.doris.analysis.Analyzer; @@ -30,6 +31,7 @@ import org.apache.doris.analysis.SqlParser; import org.apache.doris.analysis.SqlScanner; import org.apache.doris.analysis.TableName; +import org.apache.doris.analysis.TableRef; import org.apache.doris.analysis.TypeDef; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.backup.Snapshot; @@ -196,6 +198,7 @@ import org.apache.doris.thrift.TTableIndexQueryStats; import org.apache.doris.thrift.TTableMetadataNameIds; import org.apache.doris.thrift.TTableQueryStats; +import org.apache.doris.thrift.TTableRef; import org.apache.doris.thrift.TTableStatus; import org.apache.doris.thrift.TTabletLocation; import org.apache.doris.thrift.TTxnParams; @@ -2923,11 +2926,22 @@ private TRestoreSnapshotResult restoreSnapshotImpl(TRestoreSnapshotRequest reque TStatus status = new TStatus(TStatusCode.OK); result.setStatus(status); - LabelName label = new LabelName(request.getDb(), request.getLabelName()); String repoName = request.getRepoName(); Map properties = request.getProperties(); - RestoreStmt restoreStmt = new RestoreStmt(label, repoName, null, properties, request.getMeta(), + AbstractBackupTableRefClause restoreTableRefClause = null; + if (request.isSetTableRefs()) { + List tableRefs = new ArrayList<>(); + for (TTableRef tTableRef : request.getTableRefs()) { + tableRefs.add(new TableRef(new TableName(tTableRef.getTable()), tTableRef.getAliasName())); + } + + if (tableRefs.size() > 0) { + boolean isExclude = false; + restoreTableRefClause = new AbstractBackupTableRefClause(isExclude, tableRefs); + } + } + RestoreStmt restoreStmt = new RestoreStmt(label, repoName, restoreTableRefClause, properties, request.getMeta(), request.getJobInfo()); restoreStmt.setIsBeingSynced(); LOG.trace("restore snapshot info, restoreStmt: {}", restoreStmt); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java index 9689227d7c1023d..b1757a4d3f0f352 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java @@ -53,6 +53,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -150,7 +151,7 @@ void testMergeGroup() { * Group 4: Join(Group 0, Group 3) */ @Test - public void testInsertSameGroup() { + void testInsertSameGroup() { PlanChecker.from(MemoTestUtils.createConnectContext(), logicalJoinABC) .transform( // swap join's children @@ -176,7 +177,7 @@ public void testInsertSameGroup() { } @Test - public void initByOneLevelPlan() { + void initByOneLevelPlan() { OlapTable table = PlanConstructor.newOlapTable(0, "a", 1); LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), table); @@ -188,7 +189,7 @@ public void initByOneLevelPlan() { } @Test - public void initByTwoLevelChainPlan() { + void initByTwoLevelChainPlan() { Plan topProject = new LogicalPlanBuilder(scan) .project(ImmutableList.of(0)) .build(); @@ -203,7 +204,7 @@ public void initByTwoLevelChainPlan() { } @Test - public void initByJoinSameUnboundTable() { + void initByJoinSameUnboundTable() { UnboundRelation scanA = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")); // when unboundRelation contains id, the case is illegal. @@ -213,7 +214,7 @@ public void initByJoinSameUnboundTable() { } @Test - public void initByJoinSameLogicalTable() { + void initByJoinSameLogicalTable() { OlapTable tableA = PlanConstructor.newOlapTable(0, "a", 1); LogicalOlapScan scanA = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); LogicalOlapScan scanA1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); @@ -231,7 +232,7 @@ public void initByJoinSameLogicalTable() { } @Test - public void initByTwoLevelJoinPlan() { + void initByTwoLevelJoinPlan() { OlapTable tableA = PlanConstructor.newOlapTable(0, "a", 1); OlapTable tableB = PlanConstructor.newOlapTable(0, "b", 1); LogicalOlapScan scanA = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); @@ -250,7 +251,7 @@ public void initByTwoLevelJoinPlan() { } @Test - public void initByThreeLevelChainPlan() { + void initByThreeLevelChainPlan() { Set exprs = ImmutableSet.of(new EqualTo(scan.getOutput().get(0), Literal.of(1))); Plan filter = new LogicalPlanBuilder(scan) .project(ImmutableList.of(0)) @@ -269,7 +270,7 @@ public void initByThreeLevelChainPlan() { } @Test - public void initByThreeLevelBushyPlan() { + void initByThreeLevelBushyPlan() { OlapTable tableA = PlanConstructor.newOlapTable(0, "a", 1); OlapTable tableB = PlanConstructor.newOlapTable(0, "b", 1); OlapTable tableC = PlanConstructor.newOlapTable(0, "c", 1); @@ -306,7 +307,7 @@ public void initByThreeLevelBushyPlan() { * UnboundRelation(student) -> UnboundRelation(student) */ @Test - public void a2a() { + void a2a() { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); PlanChecker.from(connectContext, student) .applyBottomUpInMemo( @@ -322,7 +323,7 @@ public void a2a() { * UnboundRelation(student) -> logicalOlapScan(student) */ @Test - public void a2b() { + void a2b() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); PlanChecker.from(connectContext, new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student"))) @@ -339,7 +340,7 @@ public void a2b() { * logicalOlapScan(student) -> new logicalOlapScan(student) */ @Test - public void a2newA() { + void a2newA() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); PlanChecker.from(connectContext, student) @@ -360,7 +361,7 @@ public void a2newA() { * logicalOlapScan(student) */ @Test - public void a2bc() { + void a2bc() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, student); @@ -393,7 +394,7 @@ public void a2bc() { * the similar case is: A -> B(C(A)) */ @Test - public void a2ba() { + void a2ba() { // invalid case Assertions.assertThrows(IllegalStateException.class, () -> { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); @@ -452,7 +453,7 @@ public void a2ba() { * 2. A -> B(A(C)) */ /*@Test() - public void a2ab() { + void a2ab() { Assertions.assertThrows(IllegalStateException.class, () -> { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit = new LogicalLimit<>(1, 0, student); @@ -478,7 +479,7 @@ public void a2ab() { * logicalOlapScan(student))) */ @Test - public void a2bcd() { + void a2bcd() { LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, scan); LogicalLimit> limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, limit5); @@ -506,7 +507,7 @@ public void a2bcd() { * logicalOlapScan(student) logicalOlapScan(student) */ @Test - public void ab2a() { + void ab2a() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); @@ -530,7 +531,7 @@ public void ab2a() { * logicalOlapScan(student) logicalOlapScan(student) */ @Test - public void ab2NewA() { + void ab2NewA() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); @@ -554,7 +555,7 @@ public void ab2NewA() { * group(logicalOlapScan(student)) */ @Test - public void ab2GroupB() { + void ab2GroupB() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); @@ -576,7 +577,7 @@ public void ab2GroupB() { * logicalOlapScan(student) */ @Test - public void ab2PlanB() { + void ab2PlanB() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); @@ -598,7 +599,7 @@ public void ab2PlanB() { * UnboundRelation(StatementScopeIdGenerator.newRelationId(), student) */ @Test - public void ab2c() { + void ab2c() { UnboundRelation relation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, relation); @@ -621,7 +622,7 @@ public void ab2c() { * UnboundRelation(student) logicalOlapScan(student) */ @Test - public void ab2cd() { + void ab2cd() { UnboundRelation relation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, relation); @@ -649,7 +650,7 @@ public void ab2cd() { * logicalOlapScan(student) logicalOlapScan(student) */ @Test - public void ab2cb() { + void ab2cb() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); @@ -678,7 +679,7 @@ public void ab2cb() { * this case is invalid, same as 'a2ab'. */ @Test - public void ab2NewANewB() { + void ab2NewANewB() { Assertions.assertThrowsExactly(IllegalStateException.class, () -> { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); @@ -704,7 +705,7 @@ public void ab2NewANewB() { * this case is invalid, we can detect it because this case is similar to 'a2ba', the 'ab2cab' is similar case too */ @Test - public void ab2ba() { + void ab2ba() { Assertions.assertThrowsExactly(IllegalStateException.class, () -> { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); @@ -732,7 +733,7 @@ public void ab2ba() { * logicalOlapScan(student))) */ @Test - public void ab2cde() { + void ab2cde() { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit3 = new LogicalLimit<>(3, 0, LimitPhase.ORIGIN, student); @@ -764,7 +765,7 @@ public void ab2cde() { * logicalOlapScan(student))) logicalOlapScan(student))) */ @Test - public void abc2bac() { + void abc2bac() { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); @@ -803,7 +804,7 @@ public void abc2bac() { * logicalOlapScan(student))) */ @Test - public void abc2bc() { + void abc2bc() { UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); @@ -828,7 +829,7 @@ public void abc2bc() { } @Test - public void testRewriteBottomPlanToOnePlan() { + void testRewriteBottomPlanToOnePlan() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, student); @@ -847,7 +848,7 @@ public void testRewriteBottomPlanToOnePlan() { } @Test - public void testRewriteBottomPlanToMultiPlan() { + void testRewriteBottomPlanToMultiPlan() { LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); @@ -869,7 +870,7 @@ public void testRewriteBottomPlanToMultiPlan() { } @Test - public void testRewriteUnboundPlanToBound() { + void testRewriteUnboundPlanToBound() { UnboundRelation unboundTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("score")); LogicalOlapScan boundTable = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); @@ -891,7 +892,8 @@ public void testRewriteUnboundPlanToBound() { } @Test - public void testRecomputeLogicalProperties() { + @Disabled + void testRecomputeLogicalProperties() { UnboundRelation unboundTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("score")); LogicalLimit unboundLimit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, unboundTable); @@ -923,7 +925,7 @@ public void testRecomputeLogicalProperties() { } @Test - public void testEliminateRootWithChildGroupInTwoLevels() { + void testEliminateRootWithChildGroupInTwoLevels() { LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, scan); @@ -935,7 +937,7 @@ public void testEliminateRootWithChildGroupInTwoLevels() { } @Test - public void testEliminateRootWithChildPlanInTwoLevels() { + void testEliminateRootWithChildPlanInTwoLevels() { LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, scan); @@ -947,7 +949,7 @@ public void testEliminateRootWithChildPlanInTwoLevels() { } @Test - public void testEliminateTwoLevelsToOnePlan() { + void testEliminateTwoLevelsToOnePlan() { LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, score); @@ -967,7 +969,7 @@ public void testEliminateTwoLevelsToOnePlan() { } @Test - public void testEliminateTwoLevelsToTwoPlans() { + void testEliminateTwoLevelsToTwoPlans() { LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit1 = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, score); @@ -996,7 +998,7 @@ public void testEliminateTwoLevelsToTwoPlans() { } @Test - public void test() { + void test() { PlanChecker.from(MemoTestUtils.createConnectContext()) .analyze(new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, new LogicalJoin<>(JoinType.LEFT_OUTER_JOIN, @@ -1055,7 +1057,7 @@ public void test() { * |---UnboundRelation */ @Test - public void testRewriteMiddlePlans() { + void testRewriteMiddlePlans() { UnboundRelation unboundRelation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject insideProject = new LogicalProject<>( ImmutableList.of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))), @@ -1112,7 +1114,7 @@ public void testRewriteMiddlePlans() { * Group0: |---UnboundRelation */ @Test - public void testEliminateRootWithChildPlanThreeLevels() { + void testEliminateRootWithChildPlanThreeLevels() { UnboundRelation unboundRelation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject insideProject = new LogicalProject<>( ImmutableList.of(new SlotReference("inside", StringType.INSTANCE, true, ImmutableList.of("test"))), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java index 781c7840e4c1c45..e0e06d866d87af4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/SelectExceptTest.java @@ -31,25 +31,26 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -public class SelectExceptTest implements MemoPatternMatchSupported { +class SelectExceptTest implements MemoPatternMatchSupported { @Test - public void testExcept() { + void testExcept() { LogicalOlapScan olapScan = PlanConstructor.newLogicalOlapScan(0, "t1", 1); LogicalProject project = new LogicalProject<>( ImmutableList.of(new UnboundStar(ImmutableList.of("db", "t1"))), ImmutableList.of(new UnboundSlot("db", "t1", "id")), + false, olapScan); PlanChecker.from(MemoTestUtils.createConnectContext()) .analyze(project) .matches( logicalProject( logicalOlapScan() - ).when(proj -> proj.getExcepts().isEmpty() && proj.getProjects().size() == 1) + ).when(proj -> proj.getExcepts().size() == 1 && proj.getProjects().size() == 1) ); } @Test - public void testParse() { + void testParse() { String sql1 = "select * except(v1, v2) from t1"; PlanChecker.from(MemoTestUtils.createConnectContext()) .checkParse(sql1, (checker) -> checker.matches( diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 0eaa570cdfb68df..e58df6cb1570a9f 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1920,6 +1920,13 @@ [['sm3sum'], 'VARCHAR', ['VARCHAR', '...'], ''], [['sm3'], 'VARCHAR', ['STRING'], ''], [['sm3sum'], 'VARCHAR', ['STRING', '...'], ''], + + [['sha'], 'VARCHAR', ['VARCHAR'], ''], + [['sha'], 'VARCHAR', ['STRING'], ''], + [['sha1'], 'VARCHAR', ['VARCHAR'], ''], + [['sha1'], 'VARCHAR', ['STRING'], ''], + [['sha2'], 'VARCHAR', ['VARCHAR', 'INT'], ''], + [['sha2'], 'VARCHAR', ['STRING', 'INT'], ''], ], # geo functions diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 6c85c0290cc0140..1507076c27e4d1f 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1085,6 +1085,7 @@ struct TGetSnapshotResult { struct TTableRef { 1: optional string table + 3: optional string alias_name } struct TRestoreSnapshotRequest { diff --git a/regression-test/data/export_p0/test_export_parquet.out b/regression-test/data/export_p0/test_export_parquet.out index f7b100376123beb..8e1377bda102257 100644 --- a/regression-test/data/export_p0/test_export_parquet.out +++ b/regression-test/data/export_p0/test_export_parquet.out @@ -1,15 +1,8 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select_export1 -- 1 2017-10-01 2017-10-01T00:00 Beijing 1 1 true 1 1 1 1.1 1.1 char1 1 -2 2017-10-01 2017-10-01T00:00 Beijing 2 2 true 2 2 2 2.2 2.2 char2 2 -3 2017-10-01 2017-10-01T00:00 Beijing 3 3 true 3 3 3 3.3 3.3 char3 3 -4 2017-10-01 2017-10-01T00:00 Beijing 4 4 true 4 4 4 4.4 4.4 char4 4 -5 2017-10-01 2017-10-01T00:00 Beijing 5 5 true 5 5 5 5.5 5.5 char5 5 -6 2017-10-01 2017-10-01T00:00 Beijing 6 6 true 6 6 6 6.6 6.6 char6 6 -7 2017-10-01 2017-10-01T00:00 Beijing 7 7 true 7 7 7 7.7 7.7 char7 7 -8 2017-10-01 2017-10-01T00:00 Beijing 8 8 true 8 8 8 8.8 8.8 char8 8 -9 2017-10-01 2017-10-01T00:00 Beijing 9 9 true 9 9 9 9.9 9.9 char9 9 10 2017-10-01 2017-10-01T00:00 Beijing 10 10 true 10 10 10 10.1 10.1 char10 10 +100 2017-10-01 2017-10-01T00:00 \N \N \N \N \N \N \N \N \N \N \N 11 2017-10-01 2017-10-01T00:00 Beijing 11 11 true 11 11 11 11.11 11.11 char11 11 12 2017-10-01 2017-10-01T00:00 Beijing 12 12 true 12 12 12 12.12 12.12 char12 12 13 2017-10-01 2017-10-01T00:00 Beijing 13 13 true 13 13 13 13.13 13.13 char13 13 @@ -19,6 +12,7 @@ 17 2017-10-01 2017-10-01T00:00 Beijing 17 17 true 17 17 17 17.17 17.17 char17 17 18 2017-10-01 2017-10-01T00:00 Beijing 18 18 true 18 18 18 18.18 18.18 char18 18 19 2017-10-01 2017-10-01T00:00 Beijing 19 19 true 19 19 19 19.19 19.19 char19 19 +2 2017-10-01 2017-10-01T00:00 Beijing 2 2 true 2 2 2 2.2 2.2 char2 2 20 2017-10-01 2017-10-01T00:00 Beijing 20 20 true 20 20 20 20.2 20.2 char20 20 21 2017-10-01 2017-10-01T00:00 Beijing 21 21 true 21 21 21 21.21 21.21 char21 21 22 2017-10-01 2017-10-01T00:00 Beijing 22 22 true 22 22 22 22.22 22.22 char22 22 @@ -29,6 +23,7 @@ 27 2017-10-01 2017-10-01T00:00 Beijing 27 27 true 27 27 27 27.27 27.27 char27 27 28 2017-10-01 2017-10-01T00:00 Beijing 28 28 true 28 28 28 28.28 28.28 char28 28 29 2017-10-01 2017-10-01T00:00 Beijing 29 29 true 29 29 29 29.29 29.29 char29 29 +3 2017-10-01 2017-10-01T00:00 Beijing 3 3 true 3 3 3 3.3 3.3 char3 3 30 2017-10-01 2017-10-01T00:00 Beijing 30 30 true 30 30 30 30.3 30.3 char30 30 31 2017-10-01 2017-10-01T00:00 Beijing 31 31 true 31 31 31 31.31 31.31 char31 31 32 2017-10-01 2017-10-01T00:00 Beijing 32 32 true 32 32 32 32.32 32.32 char32 32 @@ -39,6 +34,7 @@ 37 2017-10-01 2017-10-01T00:00 Beijing 37 37 true 37 37 37 37.37 37.37 char37 37 38 2017-10-01 2017-10-01T00:00 Beijing 38 38 true 38 38 38 38.38 38.38 char38 38 39 2017-10-01 2017-10-01T00:00 Beijing 39 39 true 39 39 39 39.39 39.39 char39 39 +4 2017-10-01 2017-10-01T00:00 Beijing 4 4 true 4 4 4 4.4 4.4 char4 4 40 2017-10-01 2017-10-01T00:00 Beijing 40 40 true 40 40 40 40.4 40.4 char40 40 41 2017-10-01 2017-10-01T00:00 Beijing 41 41 true 41 41 41 41.41 41.41 char41 41 42 2017-10-01 2017-10-01T00:00 Beijing 42 42 true 42 42 42 42.42 42.42 char42 42 @@ -49,6 +45,7 @@ 47 2017-10-01 2017-10-01T00:00 Beijing 47 47 true 47 47 47 47.47 47.47 char47 47 48 2017-10-01 2017-10-01T00:00 Beijing 48 48 true 48 48 48 48.48 48.48 char48 48 49 2017-10-01 2017-10-01T00:00 Beijing 49 49 true 49 49 49 49.49 49.49 char49 49 +5 2017-10-01 2017-10-01T00:00 Beijing 5 5 true 5 5 5 5.5 5.5 char5 5 50 2017-10-01 2017-10-01T00:00 Beijing 50 50 true 50 50 50 50.5 50.5 char50 50 51 2017-10-01 2017-10-01T00:00 Beijing 51 51 true 51 51 51 51.51 51.51 char51 51 52 2017-10-01 2017-10-01T00:00 Beijing 52 52 true 52 52 52 52.52 52.52 char52 52 @@ -59,6 +56,7 @@ 57 2017-10-01 2017-10-01T00:00 Beijing 57 57 true 57 57 57 57.57 57.57 char57 57 58 2017-10-01 2017-10-01T00:00 Beijing 58 58 true 58 58 58 58.58 58.58 char58 58 59 2017-10-01 2017-10-01T00:00 Beijing 59 59 true 59 59 59 59.59 59.59 char59 59 +6 2017-10-01 2017-10-01T00:00 Beijing 6 6 true 6 6 6 6.6 6.6 char6 6 60 2017-10-01 2017-10-01T00:00 Beijing 60 60 true 60 60 60 60.6 60.6 char60 60 61 2017-10-01 2017-10-01T00:00 Beijing 61 61 true 61 61 61 61.61 61.61 char61 61 62 2017-10-01 2017-10-01T00:00 Beijing 62 62 true 62 62 62 62.62 62.62 char62 62 @@ -69,6 +67,7 @@ 67 2017-10-01 2017-10-01T00:00 Beijing 67 67 true 67 67 67 67.67 67.67 char67 67 68 2017-10-01 2017-10-01T00:00 Beijing 68 68 true 68 68 68 68.68 68.68 char68 68 69 2017-10-01 2017-10-01T00:00 Beijing 69 69 true 69 69 69 69.69 69.69 char69 69 +7 2017-10-01 2017-10-01T00:00 Beijing 7 7 true 7 7 7 7.7 7.7 char7 7 70 2017-10-01 2017-10-01T00:00 Beijing 70 70 true 70 70 70 70.7 70.7 char70 70 71 2017-10-01 2017-10-01T00:00 Beijing 71 71 true 71 71 71 71.71 71.71 char71 71 72 2017-10-01 2017-10-01T00:00 Beijing 72 72 true 72 72 72 72.72 72.72 char72 72 @@ -79,6 +78,7 @@ 77 2017-10-01 2017-10-01T00:00 Beijing 77 77 true 77 77 77 77.77 77.77 char77 77 78 2017-10-01 2017-10-01T00:00 Beijing 78 78 true 78 78 78 78.78 78.78 char78 78 79 2017-10-01 2017-10-01T00:00 Beijing 79 79 true 79 79 79 79.79 79.79 char79 79 +8 2017-10-01 2017-10-01T00:00 Beijing 8 8 true 8 8 8 8.8 8.8 char8 8 80 2017-10-01 2017-10-01T00:00 Beijing 80 80 true 80 80 80 80.8 80.8 char80 80 81 2017-10-01 2017-10-01T00:00 Beijing 81 81 true 81 81 81 81.81 81.81 char81 81 82 2017-10-01 2017-10-01T00:00 Beijing 82 82 true 82 82 82 82.82 82.82 char82 82 @@ -89,6 +89,7 @@ 87 2017-10-01 2017-10-01T00:00 Beijing 87 87 true 87 87 87 87.87 87.87 char87 87 88 2017-10-01 2017-10-01T00:00 Beijing 88 88 true 88 88 88 88.88 88.88 char88 88 89 2017-10-01 2017-10-01T00:00 Beijing 89 89 true 89 89 89 89.89 89.89 char89 89 +9 2017-10-01 2017-10-01T00:00 Beijing 9 9 true 9 9 9 9.9 9.9 char9 9 90 2017-10-01 2017-10-01T00:00 Beijing 90 90 true 90 90 90 90.9 90.9 char90 90 91 2017-10-01 2017-10-01T00:00 Beijing 91 91 true 91 91 91 91.91 91.91 char91 91 92 2017-10-01 2017-10-01T00:00 Beijing 92 92 true 92 92 92 92.92 92.92 char92 92 @@ -99,107 +100,106 @@ 97 2017-10-01 2017-10-01T00:00 Beijing 97 97 true 97 97 97 97.97 97.97 char97 97 98 2017-10-01 2017-10-01T00:00 Beijing 98 98 true 98 98 98 98.98 98.98 char98 98 99 2017-10-01 2017-10-01T00:00 Beijing 99 99 true 99 99 99 99.99 99.99 char99 99 -100 2017-10-01 2017-10-01T00:00 \N \N \N \N \N \N \N \N \N \N \N -- !select_load1 -- -1 2017-10-01 2017-10-01T00:00 Beijing 1 1 true 1 1 \N 1.1 1.1 char1 1 -2 2017-10-01 2017-10-01T00:00 Beijing 2 2 true 2 2 \N 2.2 2.2 char2 2 -3 2017-10-01 2017-10-01T00:00 Beijing 3 3 true 3 3 \N 3.3 3.3 char3 3 -4 2017-10-01 2017-10-01T00:00 Beijing 4 4 true 4 4 \N 4.4 4.4 char4 4 -5 2017-10-01 2017-10-01T00:00 Beijing 5 5 true 5 5 \N 5.5 5.5 char5 5 -6 2017-10-01 2017-10-01T00:00 Beijing 6 6 true 6 6 \N 6.6 6.6 char6 6 -7 2017-10-01 2017-10-01T00:00 Beijing 7 7 true 7 7 \N 7.7 7.7 char7 7 -8 2017-10-01 2017-10-01T00:00 Beijing 8 8 true 8 8 \N 8.8 8.8 char8 8 -9 2017-10-01 2017-10-01T00:00 Beijing 9 9 true 9 9 \N 9.9 9.9 char9 9 -10 2017-10-01 2017-10-01T00:00 Beijing 10 10 true 10 10 \N 10.1 10.1 char10 10 -11 2017-10-01 2017-10-01T00:00 Beijing 11 11 true 11 11 \N 11.11 11.11 char11 11 -12 2017-10-01 2017-10-01T00:00 Beijing 12 12 true 12 12 \N 12.12 12.12 char12 12 -13 2017-10-01 2017-10-01T00:00 Beijing 13 13 true 13 13 \N 13.13 13.13 char13 13 -14 2017-10-01 2017-10-01T00:00 Beijing 14 14 true 14 14 \N 14.14 14.14 char14 14 -15 2017-10-01 2017-10-01T00:00 Beijing 15 15 true 15 15 \N 15.15 15.15 char15 15 -16 2017-10-01 2017-10-01T00:00 Beijing 16 16 true 16 16 \N 16.16 16.16 char16 16 -17 2017-10-01 2017-10-01T00:00 Beijing 17 17 true 17 17 \N 17.17 17.17 char17 17 -18 2017-10-01 2017-10-01T00:00 Beijing 18 18 true 18 18 \N 18.18 18.18 char18 18 -19 2017-10-01 2017-10-01T00:00 Beijing 19 19 true 19 19 \N 19.19 19.19 char19 19 -20 2017-10-01 2017-10-01T00:00 Beijing 20 20 true 20 20 \N 20.2 20.2 char20 20 -21 2017-10-01 2017-10-01T00:00 Beijing 21 21 true 21 21 \N 21.21 21.21 char21 21 -22 2017-10-01 2017-10-01T00:00 Beijing 22 22 true 22 22 \N 22.22 22.22 char22 22 -23 2017-10-01 2017-10-01T00:00 Beijing 23 23 true 23 23 \N 23.23 23.23 char23 23 -24 2017-10-01 2017-10-01T00:00 Beijing 24 24 true 24 24 \N 24.24 24.24 char24 24 -25 2017-10-01 2017-10-01T00:00 Beijing 25 25 true 25 25 \N 25.25 25.25 char25 25 -26 2017-10-01 2017-10-01T00:00 Beijing 26 26 true 26 26 \N 26.26 26.26 char26 26 -27 2017-10-01 2017-10-01T00:00 Beijing 27 27 true 27 27 \N 27.27 27.27 char27 27 -28 2017-10-01 2017-10-01T00:00 Beijing 28 28 true 28 28 \N 28.28 28.28 char28 28 -29 2017-10-01 2017-10-01T00:00 Beijing 29 29 true 29 29 \N 29.29 29.29 char29 29 -30 2017-10-01 2017-10-01T00:00 Beijing 30 30 true 30 30 \N 30.3 30.3 char30 30 -31 2017-10-01 2017-10-01T00:00 Beijing 31 31 true 31 31 \N 31.31 31.31 char31 31 -32 2017-10-01 2017-10-01T00:00 Beijing 32 32 true 32 32 \N 32.32 32.32 char32 32 -33 2017-10-01 2017-10-01T00:00 Beijing 33 33 true 33 33 \N 33.33 33.33 char33 33 -34 2017-10-01 2017-10-01T00:00 Beijing 34 34 true 34 34 \N 34.34 34.34 char34 34 -35 2017-10-01 2017-10-01T00:00 Beijing 35 35 true 35 35 \N 35.35 35.35 char35 35 -36 2017-10-01 2017-10-01T00:00 Beijing 36 36 true 36 36 \N 36.36 36.36 char36 36 -37 2017-10-01 2017-10-01T00:00 Beijing 37 37 true 37 37 \N 37.37 37.37 char37 37 -38 2017-10-01 2017-10-01T00:00 Beijing 38 38 true 38 38 \N 38.38 38.38 char38 38 -39 2017-10-01 2017-10-01T00:00 Beijing 39 39 true 39 39 \N 39.39 39.39 char39 39 -40 2017-10-01 2017-10-01T00:00 Beijing 40 40 true 40 40 \N 40.4 40.4 char40 40 -41 2017-10-01 2017-10-01T00:00 Beijing 41 41 true 41 41 \N 41.41 41.41 char41 41 -42 2017-10-01 2017-10-01T00:00 Beijing 42 42 true 42 42 \N 42.42 42.42 char42 42 -43 2017-10-01 2017-10-01T00:00 Beijing 43 43 true 43 43 \N 43.43 43.43 char43 43 -44 2017-10-01 2017-10-01T00:00 Beijing 44 44 true 44 44 \N 44.44 44.44 char44 44 -45 2017-10-01 2017-10-01T00:00 Beijing 45 45 true 45 45 \N 45.45 45.45 char45 45 -46 2017-10-01 2017-10-01T00:00 Beijing 46 46 true 46 46 \N 46.46 46.46 char46 46 -47 2017-10-01 2017-10-01T00:00 Beijing 47 47 true 47 47 \N 47.47 47.47 char47 47 -48 2017-10-01 2017-10-01T00:00 Beijing 48 48 true 48 48 \N 48.48 48.48 char48 48 -49 2017-10-01 2017-10-01T00:00 Beijing 49 49 true 49 49 \N 49.49 49.49 char49 49 -50 2017-10-01 2017-10-01T00:00 Beijing 50 50 true 50 50 \N 50.5 50.5 char50 50 -51 2017-10-01 2017-10-01T00:00 Beijing 51 51 true 51 51 \N 51.51 51.51 char51 51 -52 2017-10-01 2017-10-01T00:00 Beijing 52 52 true 52 52 \N 52.52 52.52 char52 52 -53 2017-10-01 2017-10-01T00:00 Beijing 53 53 true 53 53 \N 53.53 53.53 char53 53 -54 2017-10-01 2017-10-01T00:00 Beijing 54 54 true 54 54 \N 54.54 54.54 char54 54 -55 2017-10-01 2017-10-01T00:00 Beijing 55 55 true 55 55 \N 55.55 55.55 char55 55 -56 2017-10-01 2017-10-01T00:00 Beijing 56 56 true 56 56 \N 56.56 56.56 char56 56 -57 2017-10-01 2017-10-01T00:00 Beijing 57 57 true 57 57 \N 57.57 57.57 char57 57 -58 2017-10-01 2017-10-01T00:00 Beijing 58 58 true 58 58 \N 58.58 58.58 char58 58 -59 2017-10-01 2017-10-01T00:00 Beijing 59 59 true 59 59 \N 59.59 59.59 char59 59 -60 2017-10-01 2017-10-01T00:00 Beijing 60 60 true 60 60 \N 60.6 60.6 char60 60 -61 2017-10-01 2017-10-01T00:00 Beijing 61 61 true 61 61 \N 61.61 61.61 char61 61 -62 2017-10-01 2017-10-01T00:00 Beijing 62 62 true 62 62 \N 62.62 62.62 char62 62 -63 2017-10-01 2017-10-01T00:00 Beijing 63 63 true 63 63 \N 63.63 63.63 char63 63 -64 2017-10-01 2017-10-01T00:00 Beijing 64 64 true 64 64 \N 64.64 64.64 char64 64 -65 2017-10-01 2017-10-01T00:00 Beijing 65 65 true 65 65 \N 65.65 65.65 char65 65 -66 2017-10-01 2017-10-01T00:00 Beijing 66 66 true 66 66 \N 66.66 66.66 char66 66 -67 2017-10-01 2017-10-01T00:00 Beijing 67 67 true 67 67 \N 67.67 67.67 char67 67 -68 2017-10-01 2017-10-01T00:00 Beijing 68 68 true 68 68 \N 68.68 68.68 char68 68 -69 2017-10-01 2017-10-01T00:00 Beijing 69 69 true 69 69 \N 69.69 69.69 char69 69 -70 2017-10-01 2017-10-01T00:00 Beijing 70 70 true 70 70 \N 70.7 70.7 char70 70 -71 2017-10-01 2017-10-01T00:00 Beijing 71 71 true 71 71 \N 71.71 71.71 char71 71 -72 2017-10-01 2017-10-01T00:00 Beijing 72 72 true 72 72 \N 72.72 72.72 char72 72 -73 2017-10-01 2017-10-01T00:00 Beijing 73 73 true 73 73 \N 73.73 73.73 char73 73 -74 2017-10-01 2017-10-01T00:00 Beijing 74 74 true 74 74 \N 74.74 74.74 char74 74 -75 2017-10-01 2017-10-01T00:00 Beijing 75 75 true 75 75 \N 75.75 75.75 char75 75 -76 2017-10-01 2017-10-01T00:00 Beijing 76 76 true 76 76 \N 76.76 76.76 char76 76 -77 2017-10-01 2017-10-01T00:00 Beijing 77 77 true 77 77 \N 77.77 77.77 char77 77 -78 2017-10-01 2017-10-01T00:00 Beijing 78 78 true 78 78 \N 78.78 78.78 char78 78 -79 2017-10-01 2017-10-01T00:00 Beijing 79 79 true 79 79 \N 79.79 79.79 char79 79 -80 2017-10-01 2017-10-01T00:00 Beijing 80 80 true 80 80 \N 80.8 80.8 char80 80 -81 2017-10-01 2017-10-01T00:00 Beijing 81 81 true 81 81 \N 81.81 81.81 char81 81 -82 2017-10-01 2017-10-01T00:00 Beijing 82 82 true 82 82 \N 82.82 82.82 char82 82 -83 2017-10-01 2017-10-01T00:00 Beijing 83 83 true 83 83 \N 83.83 83.83 char83 83 -84 2017-10-01 2017-10-01T00:00 Beijing 84 84 true 84 84 \N 84.84 84.84 char84 84 -85 2017-10-01 2017-10-01T00:00 Beijing 85 85 true 85 85 \N 85.85 85.85 char85 85 -86 2017-10-01 2017-10-01T00:00 Beijing 86 86 true 86 86 \N 86.86 86.86 char86 86 -87 2017-10-01 2017-10-01T00:00 Beijing 87 87 true 87 87 \N 87.87 87.87 char87 87 -88 2017-10-01 2017-10-01T00:00 Beijing 88 88 true 88 88 \N 88.88 88.88 char88 88 -89 2017-10-01 2017-10-01T00:00 Beijing 89 89 true 89 89 \N 89.89 89.89 char89 89 -90 2017-10-01 2017-10-01T00:00 Beijing 90 90 true 90 90 \N 90.9 90.9 char90 90 -91 2017-10-01 2017-10-01T00:00 Beijing 91 91 true 91 91 \N 91.91 91.91 char91 91 -92 2017-10-01 2017-10-01T00:00 Beijing 92 92 true 92 92 \N 92.92 92.92 char92 92 -93 2017-10-01 2017-10-01T00:00 Beijing 93 93 true 93 93 \N 93.93 93.93 char93 93 -94 2017-10-01 2017-10-01T00:00 Beijing 94 94 true 94 94 \N 94.94 94.94 char94 94 -95 2017-10-01 2017-10-01T00:00 Beijing 95 95 true 95 95 \N 95.95 95.95 char95 95 -96 2017-10-01 2017-10-01T00:00 Beijing 96 96 true 96 96 \N 96.96 96.96 char96 96 -97 2017-10-01 2017-10-01T00:00 Beijing 97 97 true 97 97 \N 97.97 97.97 char97 97 -98 2017-10-01 2017-10-01T00:00 Beijing 98 98 true 98 98 \N 98.98 98.98 char98 98 -99 2017-10-01 2017-10-01T00:00 Beijing 99 99 true 99 99 \N 99.99 99.99 char99 99 -100 2017-10-01 2017-10-01T00:00 \N \N \N \N \N \N \N \N \N \N \N +1 2017-10-01 2017-10-01 00:00:00 Beijing 1 1 true 1 1 1.1 1.1 char1 1 1 +10 2017-10-01 2017-10-01 00:00:00 Beijing 10 10 true 10 10 10.1 10.1 char10 10 10 +100 2017-10-01 2017-10-01 00:00:00 \N \N \N \N \N \N \N \N \N \N \N +11 2017-10-01 2017-10-01 00:00:00 Beijing 11 11 true 11 11 11.11 11.11 char11 11 11 +12 2017-10-01 2017-10-01 00:00:00 Beijing 12 12 true 12 12 12.12 12.12 char12 12 12 +13 2017-10-01 2017-10-01 00:00:00 Beijing 13 13 true 13 13 13.13 13.13 char13 13 13 +14 2017-10-01 2017-10-01 00:00:00 Beijing 14 14 true 14 14 14.14 14.14 char14 14 14 +15 2017-10-01 2017-10-01 00:00:00 Beijing 15 15 true 15 15 15.15 15.15 char15 15 15 +16 2017-10-01 2017-10-01 00:00:00 Beijing 16 16 true 16 16 16.16 16.16 char16 16 16 +17 2017-10-01 2017-10-01 00:00:00 Beijing 17 17 true 17 17 17.17 17.17 char17 17 17 +18 2017-10-01 2017-10-01 00:00:00 Beijing 18 18 true 18 18 18.18 18.18 char18 18 18 +19 2017-10-01 2017-10-01 00:00:00 Beijing 19 19 true 19 19 19.19 19.19 char19 19 19 +2 2017-10-01 2017-10-01 00:00:00 Beijing 2 2 true 2 2 2.2 2.2 char2 2 2 +20 2017-10-01 2017-10-01 00:00:00 Beijing 20 20 true 20 20 20.2 20.2 char20 20 20 +21 2017-10-01 2017-10-01 00:00:00 Beijing 21 21 true 21 21 21.21 21.21 char21 21 21 +22 2017-10-01 2017-10-01 00:00:00 Beijing 22 22 true 22 22 22.22 22.22 char22 22 22 +23 2017-10-01 2017-10-01 00:00:00 Beijing 23 23 true 23 23 23.23 23.23 char23 23 23 +24 2017-10-01 2017-10-01 00:00:00 Beijing 24 24 true 24 24 24.24 24.24 char24 24 24 +25 2017-10-01 2017-10-01 00:00:00 Beijing 25 25 true 25 25 25.25 25.25 char25 25 25 +26 2017-10-01 2017-10-01 00:00:00 Beijing 26 26 true 26 26 26.26 26.26 char26 26 26 +27 2017-10-01 2017-10-01 00:00:00 Beijing 27 27 true 27 27 27.27 27.27 char27 27 27 +28 2017-10-01 2017-10-01 00:00:00 Beijing 28 28 true 28 28 28.28 28.28 char28 28 28 +29 2017-10-01 2017-10-01 00:00:00 Beijing 29 29 true 29 29 29.29 29.29 char29 29 29 +3 2017-10-01 2017-10-01 00:00:00 Beijing 3 3 true 3 3 3.3 3.3 char3 3 3 +30 2017-10-01 2017-10-01 00:00:00 Beijing 30 30 true 30 30 30.3 30.3 char30 30 30 +31 2017-10-01 2017-10-01 00:00:00 Beijing 31 31 true 31 31 31.31 31.31 char31 31 31 +32 2017-10-01 2017-10-01 00:00:00 Beijing 32 32 true 32 32 32.32 32.32 char32 32 32 +33 2017-10-01 2017-10-01 00:00:00 Beijing 33 33 true 33 33 33.33 33.33 char33 33 33 +34 2017-10-01 2017-10-01 00:00:00 Beijing 34 34 true 34 34 34.34 34.34 char34 34 34 +35 2017-10-01 2017-10-01 00:00:00 Beijing 35 35 true 35 35 35.35 35.35 char35 35 35 +36 2017-10-01 2017-10-01 00:00:00 Beijing 36 36 true 36 36 36.36 36.36 char36 36 36 +37 2017-10-01 2017-10-01 00:00:00 Beijing 37 37 true 37 37 37.37 37.37 char37 37 37 +38 2017-10-01 2017-10-01 00:00:00 Beijing 38 38 true 38 38 38.38 38.38 char38 38 38 +39 2017-10-01 2017-10-01 00:00:00 Beijing 39 39 true 39 39 39.39 39.39 char39 39 39 +4 2017-10-01 2017-10-01 00:00:00 Beijing 4 4 true 4 4 4.4 4.4 char4 4 4 +40 2017-10-01 2017-10-01 00:00:00 Beijing 40 40 true 40 40 40.4 40.4 char40 40 40 +41 2017-10-01 2017-10-01 00:00:00 Beijing 41 41 true 41 41 41.41 41.41 char41 41 41 +42 2017-10-01 2017-10-01 00:00:00 Beijing 42 42 true 42 42 42.42 42.42 char42 42 42 +43 2017-10-01 2017-10-01 00:00:00 Beijing 43 43 true 43 43 43.43 43.43 char43 43 43 +44 2017-10-01 2017-10-01 00:00:00 Beijing 44 44 true 44 44 44.44 44.44 char44 44 44 +45 2017-10-01 2017-10-01 00:00:00 Beijing 45 45 true 45 45 45.45 45.45 char45 45 45 +46 2017-10-01 2017-10-01 00:00:00 Beijing 46 46 true 46 46 46.46 46.46 char46 46 46 +47 2017-10-01 2017-10-01 00:00:00 Beijing 47 47 true 47 47 47.47 47.47 char47 47 47 +48 2017-10-01 2017-10-01 00:00:00 Beijing 48 48 true 48 48 48.48 48.48 char48 48 48 +49 2017-10-01 2017-10-01 00:00:00 Beijing 49 49 true 49 49 49.49 49.49 char49 49 49 +5 2017-10-01 2017-10-01 00:00:00 Beijing 5 5 true 5 5 5.5 5.5 char5 5 5 +50 2017-10-01 2017-10-01 00:00:00 Beijing 50 50 true 50 50 50.5 50.5 char50 50 50 +51 2017-10-01 2017-10-01 00:00:00 Beijing 51 51 true 51 51 51.51 51.51 char51 51 51 +52 2017-10-01 2017-10-01 00:00:00 Beijing 52 52 true 52 52 52.52 52.52 char52 52 52 +53 2017-10-01 2017-10-01 00:00:00 Beijing 53 53 true 53 53 53.53 53.53 char53 53 53 +54 2017-10-01 2017-10-01 00:00:00 Beijing 54 54 true 54 54 54.54 54.54 char54 54 54 +55 2017-10-01 2017-10-01 00:00:00 Beijing 55 55 true 55 55 55.55 55.55 char55 55 55 +56 2017-10-01 2017-10-01 00:00:00 Beijing 56 56 true 56 56 56.56 56.56 char56 56 56 +57 2017-10-01 2017-10-01 00:00:00 Beijing 57 57 true 57 57 57.57 57.57 char57 57 57 +58 2017-10-01 2017-10-01 00:00:00 Beijing 58 58 true 58 58 58.58 58.58 char58 58 58 +59 2017-10-01 2017-10-01 00:00:00 Beijing 59 59 true 59 59 59.59 59.59 char59 59 59 +6 2017-10-01 2017-10-01 00:00:00 Beijing 6 6 true 6 6 6.6 6.6 char6 6 6 +60 2017-10-01 2017-10-01 00:00:00 Beijing 60 60 true 60 60 60.6 60.6 char60 60 60 +61 2017-10-01 2017-10-01 00:00:00 Beijing 61 61 true 61 61 61.61 61.61 char61 61 61 +62 2017-10-01 2017-10-01 00:00:00 Beijing 62 62 true 62 62 62.62 62.62 char62 62 62 +63 2017-10-01 2017-10-01 00:00:00 Beijing 63 63 true 63 63 63.63 63.63 char63 63 63 +64 2017-10-01 2017-10-01 00:00:00 Beijing 64 64 true 64 64 64.64 64.64 char64 64 64 +65 2017-10-01 2017-10-01 00:00:00 Beijing 65 65 true 65 65 65.65 65.65 char65 65 65 +66 2017-10-01 2017-10-01 00:00:00 Beijing 66 66 true 66 66 66.66 66.66 char66 66 66 +67 2017-10-01 2017-10-01 00:00:00 Beijing 67 67 true 67 67 67.67 67.67 char67 67 67 +68 2017-10-01 2017-10-01 00:00:00 Beijing 68 68 true 68 68 68.68 68.68 char68 68 68 +69 2017-10-01 2017-10-01 00:00:00 Beijing 69 69 true 69 69 69.69 69.69 char69 69 69 +7 2017-10-01 2017-10-01 00:00:00 Beijing 7 7 true 7 7 7.7 7.7 char7 7 7 +70 2017-10-01 2017-10-01 00:00:00 Beijing 70 70 true 70 70 70.7 70.7 char70 70 70 +71 2017-10-01 2017-10-01 00:00:00 Beijing 71 71 true 71 71 71.71 71.71 char71 71 71 +72 2017-10-01 2017-10-01 00:00:00 Beijing 72 72 true 72 72 72.72 72.72 char72 72 72 +73 2017-10-01 2017-10-01 00:00:00 Beijing 73 73 true 73 73 73.73 73.73 char73 73 73 +74 2017-10-01 2017-10-01 00:00:00 Beijing 74 74 true 74 74 74.74 74.74 char74 74 74 +75 2017-10-01 2017-10-01 00:00:00 Beijing 75 75 true 75 75 75.75 75.75 char75 75 75 +76 2017-10-01 2017-10-01 00:00:00 Beijing 76 76 true 76 76 76.76 76.76 char76 76 76 +77 2017-10-01 2017-10-01 00:00:00 Beijing 77 77 true 77 77 77.77 77.77 char77 77 77 +78 2017-10-01 2017-10-01 00:00:00 Beijing 78 78 true 78 78 78.78 78.78 char78 78 78 +79 2017-10-01 2017-10-01 00:00:00 Beijing 79 79 true 79 79 79.79 79.79 char79 79 79 +8 2017-10-01 2017-10-01 00:00:00 Beijing 8 8 true 8 8 8.8 8.8 char8 8 8 +80 2017-10-01 2017-10-01 00:00:00 Beijing 80 80 true 80 80 80.8 80.8 char80 80 80 +81 2017-10-01 2017-10-01 00:00:00 Beijing 81 81 true 81 81 81.81 81.81 char81 81 81 +82 2017-10-01 2017-10-01 00:00:00 Beijing 82 82 true 82 82 82.82 82.82 char82 82 82 +83 2017-10-01 2017-10-01 00:00:00 Beijing 83 83 true 83 83 83.83 83.83 char83 83 83 +84 2017-10-01 2017-10-01 00:00:00 Beijing 84 84 true 84 84 84.84 84.84 char84 84 84 +85 2017-10-01 2017-10-01 00:00:00 Beijing 85 85 true 85 85 85.85 85.85 char85 85 85 +86 2017-10-01 2017-10-01 00:00:00 Beijing 86 86 true 86 86 86.86 86.86 char86 86 86 +87 2017-10-01 2017-10-01 00:00:00 Beijing 87 87 true 87 87 87.87 87.87 char87 87 87 +88 2017-10-01 2017-10-01 00:00:00 Beijing 88 88 true 88 88 88.88 88.88 char88 88 88 +89 2017-10-01 2017-10-01 00:00:00 Beijing 89 89 true 89 89 89.89 89.89 char89 89 89 +9 2017-10-01 2017-10-01 00:00:00 Beijing 9 9 true 9 9 9.9 9.9 char9 9 9 +90 2017-10-01 2017-10-01 00:00:00 Beijing 90 90 true 90 90 90.9 90.9 char90 90 90 +91 2017-10-01 2017-10-01 00:00:00 Beijing 91 91 true 91 91 91.91 91.91 char91 91 91 +92 2017-10-01 2017-10-01 00:00:00 Beijing 92 92 true 92 92 92.92 92.92 char92 92 92 +93 2017-10-01 2017-10-01 00:00:00 Beijing 93 93 true 93 93 93.93 93.93 char93 93 93 +94 2017-10-01 2017-10-01 00:00:00 Beijing 94 94 true 94 94 94.94 94.94 char94 94 94 +95 2017-10-01 2017-10-01 00:00:00 Beijing 95 95 true 95 95 95.95 95.95 char95 95 95 +96 2017-10-01 2017-10-01 00:00:00 Beijing 96 96 true 96 96 96.96 96.96 char96 96 96 +97 2017-10-01 2017-10-01 00:00:00 Beijing 97 97 true 97 97 97.97 97.97 char97 97 97 +98 2017-10-01 2017-10-01 00:00:00 Beijing 98 98 true 98 98 98.98 98.98 char98 98 98 +99 2017-10-01 2017-10-01 00:00:00 Beijing 99 99 true 99 99 99.99 99.99 char99 99 99 diff --git a/regression-test/data/export_p0/test_export_view.out b/regression-test/data/export_p0/test_export_view.out index 2f06e611d0b5512..b3d2deba7341824 100644 --- a/regression-test/data/export_p0/test_export_view.out +++ b/regression-test/data/export_p0/test_export_view.out @@ -187,6 +187,10 @@ wangwu 80 137 3980 xiexie 150 \N 150 zhangsan 31 67 760 +-- !select_load8 -- +fangfang 120 \N 4035 +zhangsan 31 \N 760 + -- !select_load7 -- fangfang 120 167 4035 lisi 50 97 1935 @@ -194,7 +198,3 @@ wangwu 80 137 3980 xiexie 150 \N 150 zhangsan 31 67 760 --- !select_load8 -- -fangfang 120 \N 4035 -zhangsan 31 \N 760 - diff --git a/regression-test/data/external_table_p0/jdbc/test_oracle_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_oracle_jdbc_catalog.out index f239053cba8b95b..ad2572ddb35b044 100644 --- a/regression-test/data/external_table_p0/jdbc/test_oracle_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_oracle_jdbc_catalog.out @@ -131,6 +131,12 @@ DORIS_TEST 2 -99 -9999 -999999999 -999999999999999999 -999 -99999 -9999999999 -9999999999999999999 3 10 100 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 +-- !lower_case_table_names4 -- +1 alice 20 99.5 +2 bob 21 90.5 +3 jerry 23 88.0 +4 andy 21 93.0 + -- !query_clob -- 10010 liantong 10086 yidong diff --git a/regression-test/data/query_p0/sql_functions/encryption_digest/test_digest.out b/regression-test/data/query_p0/sql_functions/encryption_digest/test_digest.out new file mode 100644 index 000000000000000..56d37cf1ea925e8 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/encryption_digest/test_digest.out @@ -0,0 +1,39 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !md5 -- +68934a3e9455fa72420237eb05902327 +68934a3e9455fa72420237eb05902327 +b326b5062b2f0e69046810717534cb09 + +-- !sha1_1 -- +40bd001563085fc35165329ea1ff5c5ecbdbbeef + +-- !sha1_2 -- +01b61f61abb0108346025d913eba241ec600da51 01b61f61abb0108346025d913eba241ec600da51 +e20afd4c8d2f7966348494bc4a8b3ba2406479c3 e20afd4c8d2f7966348494bc4a8b3ba2406479c3 +ecfb6e9d7b67d81c6f4a11c6df1bd5cc7d38d21e ecfb6e9d7b67d81c6f4a11c6df1bd5cc7d38d21e + +-- !sha1_3 -- +da39a3ee5e6b4b0d3255bfef95601890afd80709 + +-- !sha1_4 -- +\N + +-- !sha2_1 -- +be504c074c5a9e0a461d8dd6e1160ac8343e5d70bbd05082868e09edc9b17c50 +86aa74b42da827c630e4a3efaeb660823af67e540828576e8413e31ac95fd494 +86d137f8ee3da44ded006c55a3b6f3460121711df8a789bf146a6d2ce3995311 + +-- !sha2_2 -- +10a699b0850fad1c219f933f821e472630eac91cc34b2e8d254b2326ab8ca025ea9fdaa1db5e7b4c1d6b84a525524f8f3100e9847e1504b199728ddf0ce3e134 +a0296b74d1f31c0ce956f793efeb003c5e987b8f4546c8a02dcadf19073ff717d86fca57bdd0f532067b4757a7d0b924bcf4c471c05f671d775bac4f00638152 +6f2c68ec336dab0e4eb4746053bd4040b181cf75a4189dff729c6ac9a8e5062440bf5b6dc0f8ae3ed8859e1aa2a55626b1ae36191c34d41362766bcac9bc2ace + +-- !sha2_3 -- +23097d223405d8228642a477bda255b32aadbce4bda0b3f7e36c9da7 + +-- !sha2_4 -- +cb00753f45a35e8bb5a03d699ac65007272c32ab0eded1631a8b605a43ff5bed8086072ba1e7cc2358baeca134c825a7 + +-- !sha2_5 -- +\N + diff --git a/regression-test/data/query_p0/sql_functions/encryption_digest/test_md5.out b/regression-test/data/query_p0/sql_functions/encryption_digest/test_md5.out deleted file mode 100644 index 9bfed7153faf7a5..000000000000000 --- a/regression-test/data/query_p0/sql_functions/encryption_digest/test_md5.out +++ /dev/null @@ -1,6 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select -- -68934a3e9455fa72420237eb05902327 -68934a3e9455fa72420237eb05902327 -b326b5062b2f0e69046810717534cb09 - diff --git a/regression-test/data/schema_change_p0/test_alter_table_column.out b/regression-test/data/schema_change_p0/test_alter_table_column.out index 81a715c2bcdd22b..f01d5d28edc7306 100644 --- a/regression-test/data/schema_change_p0/test_alter_table_column.out +++ b/regression-test/data/schema_change_p0/test_alter_table_column.out @@ -21,9 +21,9 @@ value2 INT Yes false \N SUM -- !sql -- k1 INT Yes true \N value1 INT Yes false \N NONE -value2 ARRAY Yes false [] NONE -value3 ARRAY Yes false \N NONE -value4 ARRAY No false [] NONE +value2 ARRAY Yes false [] NONE +value3 ARRAY Yes false \N NONE +value4 ARRAY No false [] NONE -- !sql -- 1 2 [] \N [] diff --git a/regression-test/pipeline/p0/conf/regression-conf.groovy b/regression-test/pipeline/p0/conf/regression-conf.groovy index 0a680eb0b8578b4..423c9f2f659744c 100644 --- a/regression-test/pipeline/p0/conf/regression-conf.groovy +++ b/regression-test/pipeline/p0/conf/regression-conf.groovy @@ -55,7 +55,7 @@ testDirectories = "" excludeGroups = "" // this suites will not be executed -excludeSuites = "test_sql_block_rule,test_ddl,test_analyze,test_leading,test_stream_load_move_memtable,test_profile,test_broker_load,test_spark_load,test_refresh_mtmv,test_bitmap_filter,test_export_parquet,nereids_delete_mow_partial_update" +excludeSuites = "test_sql_block_rule,test_ddl,test_analyze,test_leading,test_stream_load_move_memtable,test_profile,test_broker_load,test_spark_load,test_refresh_mtmv,test_bitmap_filter,nereids_delete_mow_partial_update" // this directories will not be executed excludeDirectories = "workload_manager_p1" diff --git a/regression-test/suites/export_p0/test_export_parquet.groovy b/regression-test/suites/export_p0/test_export_parquet.groovy index 100d1304b81c85f..ff8d7f6a0dbc0af 100644 --- a/regression-test/suites/export_p0/test_export_parquet.groovy +++ b/regression-test/suites/export_p0/test_export_parquet.groovy @@ -27,37 +27,16 @@ suite("test_export_parquet", "p0") { sql """ set enable_fallback_to_original_planner=false """ - // check whether the FE config 'enable_outfile_to_local' is true - StringBuilder strBuilder = new StringBuilder() - strBuilder.append("curl --location-trusted -u " + context.config.jdbcUser + ":" + context.config.jdbcPassword) - strBuilder.append(" http://" + context.config.feHttpAddress + "/rest/v1/config/fe") + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); - String command = strBuilder.toString() - def process = command.toString().execute() - def code = process.waitFor() - def err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream()))); - def out = process.getText() - logger.info("Request FE Config: code=" + code + ", out=" + out + ", err=" + err) - assertEquals(code, 0) - def response = parseJson(out.trim()) - assertEquals(response.code, 0) - assertEquals(response.msg, "success") - def configJson = response.data.rows - boolean enableOutfileToLocal = false - for (Object conf: configJson) { - assert conf instanceof Map - if (((Map) conf).get("Name").toLowerCase() == "enable_outfile_to_local") { - enableOutfileToLocal = ((Map) conf).get("Value").toLowerCase() == "true" - } - } - if (!enableOutfileToLocal) { - logger.warn("Please set enable_outfile_to_local to true to run test_outfile") - return - } def table_export_name = "test_export_parquet" def table_load_name = "test_load_parquet" - def outfile_path_prefix = """/tmp/test_export""" + def outfile_path_prefix = """${bucket}/export/p0/parquet/exp""" // create table and insert sql """ DROP TABLE IF EXISTS ${table_export_name} """ @@ -95,40 +74,19 @@ suite("test_export_parquet", "p0") { """ def insert_res = sql "show last insert;" logger.info("insert result: " + insert_res.toString()) - qt_select_export1 """ SELECT * FROM ${table_export_name} t ORDER BY user_id; """ - - - def check_path_exists = { dir_path -> - File path = new File(dir_path) - if (!path.exists()) { - assert path.mkdirs() - } else { - throw new IllegalStateException("""${dir_path} already exists! """) - } - } + order_qt_select_export1 """ SELECT * FROM ${table_export_name} t ORDER BY user_id; """ - def check_file_amounts = { dir_path, amount -> - File path = new File(dir_path) - File[] files = path.listFiles() - assert files.length == amount - } - - def delete_files = { dir_path -> - File path = new File(dir_path) - if (path.exists()) { - for (File f: path.listFiles()) { - f.delete(); - } - path.delete(); - } - } def waiting_export = { export_label -> while (true) { def res = sql """ show export where label = "${export_label}" """ logger.info("export state: " + res[0][2]) if (res[0][2] == "FINISHED") { - break; + def json = parseJson(res[0][11]) + assert json instanceof List + assertEquals("1", json.fileNumber[0][0]) + log.info("outfile_path: ${json.url[0][0]}") + return json.url[0][0]; } else if (res[0][2] == "CANCELLED") { throw new IllegalStateException("""export failed: ${res[0][10]}""") } else { @@ -142,75 +100,33 @@ suite("test_export_parquet", "p0") { def outFilePath = """${outfile_path_prefix}_${uuid}""" def label = "label_${uuid}" try { - // check export path - check_path_exists.call("${outFilePath}") - // exec export sql """ - EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/" + EXPORT TABLE ${table_export_name} TO "s3://${outFilePath}/" PROPERTIES( "label" = "${label}", "format" = "parquet", - 'columns' = 'user_id, date, datetime, city, age, sex, bool_col, int_col, bigint_col, float_col, double_col, char_col, decimal_col' + 'columns' = 'user_id, date, datetime, city, age, sex, bool_col, int_col, bigint_col, float_col, double_col, char_col, decimal_col, largeint_col' + ) + WITH S3( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" ); """ - waiting_export.call(label) + def outfile_url = waiting_export.call(label) - // check file amounts - check_file_amounts.call("${outFilePath}", 1) - - // check data correctness - sql """ DROP TABLE IF EXISTS ${table_load_name} """ - sql """ - CREATE TABLE IF NOT EXISTS ${table_load_name} ( - `user_id` INT NOT NULL COMMENT "用户id", - `date` DATE NOT NULL COMMENT "数据灌入日期时间", - `datetime` DATETIME NOT NULL COMMENT "数据灌入日期时间", - `city` VARCHAR(20) COMMENT "用户所在城市", - `age` SMALLINT COMMENT "用户年龄", - `sex` TINYINT COMMENT "用户性别", - `bool_col` boolean COMMENT "", - `int_col` int COMMENT "", - `bigint_col` bigint COMMENT "", - `largeint_col` largeint COMMENT "", - `float_col` float COMMENT "", - `double_col` double COMMENT "", - `char_col` CHAR(10) COMMENT "", - `decimal_col` decimal COMMENT "" - ) - DISTRIBUTED BY HASH(user_id) PROPERTIES("replication_num" = "1"); - """ - - File[] files = new File("${outFilePath}").listFiles() - String file_path = files[0].getAbsolutePath() - streamLoad { - table "${table_load_name}" - - set 'columns', 'user_id, date, datetime, city, age, sex, bool_col, int_col, bigint_col, float_col, double_col, char_col, decimal_col' - set 'strict_mode', 'true' - set 'format', 'parquet' - - file "${file_path}" - time 10000 // limit inflight 10s - - check { result, exception, startTime, endTime -> - if (exception != null) { - throw exception - } - log.info("Stream load result: ${result}".toString()) - def json = parseJson(result) - assertEquals("success", json.Status.toLowerCase()) - assertEquals(100, json.NumberTotalRows) - assertEquals(0, json.NumberFilteredRows) - } - } - - qt_select_load1 """ SELECT * FROM ${table_load_name} t ORDER BY user_id; """ + order_qt_select_load1 """ select * from s3( + "uri" = "http://${s3_endpoint}${outfile_url.substring(4)}0.parquet", + "s3.access_key"= "${ak}", + "s3.secret_key" = "${sk}", + "format" = "parquet", + "region" = "${region}" + ) ORDER BY user_id; + """ } finally { - try_sql("DROP TABLE IF EXISTS ${table_load_name}") - delete_files.call("${outFilePath}") + try_sql("DROP TABLE IF EXISTS ${table_export_name}") } - - try_sql("DROP TABLE IF EXISTS ${table_export_name}") } diff --git a/regression-test/suites/export_p0/test_export_view.groovy b/regression-test/suites/export_p0/test_export_view.groovy index 29bc0e96c6272a2..1f38488d7ba8410 100644 --- a/regression-test/suites/export_p0/test_export_view.groovy +++ b/regression-test/suites/export_p0/test_export_view.groovy @@ -105,7 +105,7 @@ suite("test_export_view", "p0") { def table_export_name = "test_export_base_table" def table_export_view_name = "test_export_view_table" - def table_load_name = "test_load_basic" + def table_load_name = "test_load_view_basic" def outfile_path_prefix = """/tmp/test_export""" // create table and insert @@ -339,7 +339,7 @@ suite("test_export_view", "p0") { } } - order_qt_select_load3 """ SELECT * FROM ${table_load_name} t; """ + order_qt_select_load3 """ SELECT * FROM ${table_load_name}; """ } finally { try_sql("DROP TABLE IF EXISTS ${table_load_name}") @@ -525,7 +525,7 @@ suite("test_export_view", "p0") { } - // 7. test parquet type + // 8. test orc type, where clause and columns property uuid = UUID.randomUUID().toString() outFilePath = """${outfile_path_prefix}_${uuid}""" label = "label_${uuid}" @@ -535,15 +535,17 @@ suite("test_export_view", "p0") { // exec export sql """ - EXPORT TABLE ${table_export_view_name} TO "file://${outFilePath}/" + EXPORT TABLE ${table_export_view_name} where s1 = 'fangfang' OR k1 = 31 + TO "file://${outFilePath}/" PROPERTIES( "label" = "${label}", - "format" = "parquet", + "format" = "orc", "max_file_size" = "512MB", "parallelISM" = "5", - "column_separator"="," + "columns" = "k3, s1, k1" ); """ + waiting_export.call(label) // check file amounts @@ -558,7 +560,8 @@ suite("test_export_view", "p0") { table "${table_load_name}" set 'strict_mode', 'true' - set 'format', 'parquet' + set 'columns', 'k3, s1, k1' + set 'format', 'orc' file "${file_path}" time 10000 // limit inflight 10s @@ -570,75 +573,74 @@ suite("test_export_view", "p0") { log.info("Stream load result: ${result}".toString()) def json = parseJson(result) assertEquals("success", json.Status.toLowerCase()) - assertEquals(5, json.NumberTotalRows) + assertEquals(2, json.NumberTotalRows) assertEquals(0, json.NumberFilteredRows) } } - order_qt_select_load7 """ SELECT * FROM ${table_load_name} t; """ + order_qt_select_load8 """ SELECT * FROM ${table_load_name} t; """ } finally { try_sql("DROP TABLE IF EXISTS ${table_load_name}") delete_files.call("${outFilePath}") } - // 8. test orc type, where clause and columns property + + // 7. test parquet type use s3 uuid = UUID.randomUUID().toString() - outFilePath = """${outfile_path_prefix}_${uuid}""" label = "label_${uuid}" try { - // check export path - check_path_exists.call("${outFilePath}") + + String ak = getS3AK() + String sk = getS3SK() + String s3_endpoint = getS3Endpoint() + String region = getS3Region() + String bucket = context.config.otherConfigs.get("s3BucketName"); + + outFilePath = """${bucket}/export/p0/view/parquet""" // exec export sql """ - EXPORT TABLE ${table_export_view_name} where s1 = 'fangfang' OR k1 = 31 - TO "file://${outFilePath}/" + EXPORT TABLE ${table_export_view_name} TO "s3://${outFilePath}/" PROPERTIES( "label" = "${label}", - "format" = "orc", - "max_file_size" = "512MB", - "parallelISM" = "5", - "columns" = "k3, s1, k1" + "format" = "parquet" + ) + WITH S3( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" ); """ - waiting_export.call(label) - - // check file amounts - check_file_amounts.call("${outFilePath}", 1) - - // check data correctness - create_load_table(table_load_name) - - File[] files = new File("${outFilePath}").listFiles() - String file_path = files[0].getAbsolutePath() - streamLoad { - table "${table_load_name}" - - set 'strict_mode', 'true' - set 'columns', 'k3, s1, k1' - set 'format', 'orc' - - file "${file_path}" - time 10000 // limit inflight 10s - - check { result, exception, startTime, endTime -> - if (exception != null) { - throw exception - } - log.info("Stream load result: ${result}".toString()) - def json = parseJson(result) - assertEquals("success", json.Status.toLowerCase()) - assertEquals(2, json.NumberTotalRows) - assertEquals(0, json.NumberFilteredRows) + def outfile_url = "" + while (true) { + def res = sql """ show export where label = "${label}" """ + logger.info("export state: " + res[0][2]) + if (res[0][2] == "FINISHED") { + def json = parseJson(res[0][11]) + assert json instanceof List + assertEquals("1", json.fileNumber[0][0]) + log.info("outfile_path: ${json.url[0][0]}") + outfile_url = json.url[0][0]; + break; + } else if (res[0][2] == "CANCELLED") { + throw new IllegalStateException("""export failed: ${res[0][10]}""") + } else { + sleep(5000) } } - order_qt_select_load8 """ SELECT * FROM ${table_load_name} t; """ - + order_qt_select_load7 """ select * from s3( + "uri" = "http://${s3_endpoint}${outfile_url.substring(4)}0.parquet", + "s3.access_key"= "${ak}", + "s3.secret_key" = "${sk}", + "format" = "parquet", + "region" = "${region}" + ); + """ } finally { try_sql("DROP TABLE IF EXISTS ${table_load_name}") - delete_files.call("${outFilePath}") } -} \ No newline at end of file +} diff --git a/regression-test/suites/external_table_p0/export/test_export_external_table.groovy b/regression-test/suites/external_table_p0/export/test_export_external_table.groovy index adbabe6340a99f8..48cfb9e83e8abc4 100644 --- a/regression-test/suites/external_table_p0/export/test_export_external_table.groovy +++ b/regression-test/suites/external_table_p0/export/test_export_external_table.groovy @@ -91,9 +91,9 @@ suite("test_export_external_table", "p0,external,mysql,external_docker,external_ } - + // this table name must be `test1`, because this is an external table. def table_export_name = "test1" - def table_load_name = "test_load_basic" + def table_load_name = "test_load_external__basic" def outfile_path_prefix = """/tmp/test_export""" String enabled = context.config.otherConfigs.get("enableJdbcTest") diff --git a/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy index 5db86c3dd4f57fd..79e4367793edf7b 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_oracle_jdbc_catalog.groovy @@ -147,6 +147,9 @@ suite("test_oracle_jdbc_catalog", "p0,external,oracle,external_docker,external_d qt_lower_case_table_names2 """ select * from test_char order by ID; """ qt_lower_case_table_names3 """ select * from test_int order by ID; """ + // test lower case name + order_qt_lower_case_table_names4 """ select * from student2 order by id; """ + sql """drop catalog if exists ${catalog_name} """ // test for clob type diff --git a/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy b/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy index a8fc2d5868badff..f0337fcfc7f346e 100644 --- a/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy +++ b/regression-test/suites/pipelineX/test_distinct_streaming_agg_operator.groovy @@ -61,7 +61,7 @@ suite("test_distinct_streaming_agg_operator") { select k6 from baseall group by k6 order by k6; """ - sql"""set experimental_enable_pipeline_x_engine=false; """ + sql"""set experimental_enable_pipeline_x_engine=true; """ qt_pipelineX_1 """ select * from ( select k1,k2,k3 from baseall union select k1,k2,k3 from baseall) as t ORDER BY 1, 2,3; diff --git a/regression-test/suites/pipelineX/test_repeat_operator.groovy b/regression-test/suites/pipelineX/test_repeat_operator.groovy index cb164aab0983c86..57357b0b4a6b3e5 100644 --- a/regression-test/suites/pipelineX/test_repeat_operator.groovy +++ b/regression-test/suites/pipelineX/test_repeat_operator.groovy @@ -70,7 +70,7 @@ suite("test_repeat_operator") { ORDER BY k1, k2,k3; """ - sql"""set experimental_enable_pipeline_x_engine=false; """ + sql"""set experimental_enable_pipeline_x_engine=true; """ qt_pipelineX """ SELECT k1, k2 diff --git a/regression-test/suites/pipelineX/test_union_operator.groovy b/regression-test/suites/pipelineX/test_union_operator.groovy index 7f606cfa7f57a8a..59448c35ec6a404 100644 --- a/regression-test/suites/pipelineX/test_union_operator.groovy +++ b/regression-test/suites/pipelineX/test_union_operator.groovy @@ -80,7 +80,7 @@ suite("test_union_operator") { """ - sql"""set experimental_enable_pipeline_x_engine=false,parallel_pipeline_task_num = 8;; """ + sql"""set experimental_enable_pipeline_x_engine=true,parallel_pipeline_task_num = 8;; """ qt_pipelineX """ SELECT count(*) diff --git a/regression-test/suites/query_p0/sql_functions/encryption_digest/test_md5.groovy b/regression-test/suites/query_p0/sql_functions/encryption_digest/test_digest.groovy similarity index 53% rename from regression-test/suites/query_p0/sql_functions/encryption_digest/test_md5.groovy rename to regression-test/suites/query_p0/sql_functions/encryption_digest/test_digest.groovy index 152baae5a049458..ebd9bd7d1db7a52 100644 --- a/regression-test/suites/query_p0/sql_functions/encryption_digest/test_md5.groovy +++ b/regression-test/suites/query_p0/sql_functions/encryption_digest/test_digest.groovy @@ -15,6 +15,21 @@ // specific language governing permissions and limitations // under the License. -suite("test_md5") { - qt_select "select md5(k6) from test_query_db.test order by k6" +suite("test_digest") { + qt_md5 "select md5(k6) from test_query_db.test order by k6" + qt_sha1_1 "select sha1(\"123\")" + qt_sha1_2 "select sha(k7), sha1(k7) from test_query_db.test order by k7" + qt_sha1_3 "select sha1(\"\")" + qt_sha1_4 "select sha1(NULL)" + qt_sha2_1 "select sha2(k7, 256) from test_query_db.test order by k7" + qt_sha2_2 "select sha2(k7, 512) from test_query_db.test order by k7" + qt_sha2_3 "select sha2('abc', 224)" + qt_sha2_4 "select sha2('abc', 384)" + qt_sha2_5 "select sha2(NULL, 384)" + + try { + result = sql """ select sha2("123", 255) """ + } catch (Exception e) { + assertTrue(e.getMessage().contains("only support 224/256/384/512")) + } } diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql index 67aa90696de289d..c3d19b67a2889ea 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE LO_ORDERDATE >= 19930101 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql index d3d9b16cc7bbe8a..6ab6ceea340c88a 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q1.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE LO_ORDERDATE >= 19940101 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql index b22efea51b01efd..70796c2a954cff2 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q1.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE weekofyear(LO_ORDERDATE) = 6 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql index 7dbab32f964f31b..57f2ada2964909f 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql index 884a7288fd31793..9b7a5db502f6be5 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql index 7180f48a4a41efd..3a8a5e74d48f7f3 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql index 927e504328a72b9..6b3257f1f3b2390 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ C_NATION, S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql index 63ac7953926c4b5..fefe727da82d182 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql index 25430249b0bf2ec..c4560b701edfdb9 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql index de0750590096a9a..4ae5d956e48e178 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.4 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql index 263b9e55e4da32c..87b29bf160acde3 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql index 587eeef2622ae32..8ea28f3f120d5f5 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, S_NATION, P_CATEGORY, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql index fadb807c0b01fdc..0f7c7401ab630e4 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, S_CITY, P_BRAND, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql index 934377807c60eb0..50b50bc3683925f 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql index a86974c8a1b4f80..77c026201676818 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql index 41ac021e963856b..0052db0aac34603 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql index b4777810370e046..a47ec82b5170283 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql index fc95bc2d87a3671..9ab1a95d4d73d21 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql index 20f6baf359ebd4c..b7e6bd784098eca 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql index daead15361251ec..85c470b70860e5f 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_nation, s_nation, d_year, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_nation, s_nation, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql index 1b729ea6f8c42c7..cd0b320f87ff9aa 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_city, s_city, d_year, sum(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, sum(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql index 90450d3729bfb3e..89765c02d9c6c11 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_city, s_city, d_year, SUM(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql index c59e9f10b03d629..5cef87a3fe33500 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ c_city, s_city, d_year, SUM(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql index 767af86aba2d55c..3e0227c2ea96f25 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ d_year, c_nation, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, c_nation, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql index 8accfc8e5a44de6..1338e780ae52ca9 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ d_year, s_nation, p_category, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_nation, p_category, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql index f5411faaa42707d..d8e6f7c42d40741 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ d_year, s_city, p_brand, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_city, p_brand, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql index 78b560ff246d249..ded6754a971c209 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql @@ -1,5 +1,5 @@ -- tables: lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql index 510c3d117ddec84..f102f7504d7f827 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,partsupp,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ s_acctbal, s_name, n_name, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql index 360e51763fdcf69..8bd60f0e0799b54 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS revenue, o_orderdate, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql index 99a1213d5c6f7b8..3f440947290b904 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql @@ -1,5 +1,5 @@ -- tables: orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ o_orderpriority, count(*) AS order_count FROM orders diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql index 871c3cc84c26cea..ed179f8b869be80 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem,supplier,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql index b08a070250ac56f..2dd86f8c2ce55e4 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql @@ -1,6 +1,6 @@ -- tables: lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ sum(l_extendedprice * l_discount) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * l_discount) AS revenue FROM lineitem WHERE diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql index 77c13aa4e8cacc7..6453c1094a836d9 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql @@ -1,5 +1,5 @@ -- tables: supplier,lineitem,orders,customer,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ supp_nation, cust_nation, l_year, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql index bbdb4fea5822e54..e4c46fb0847bba6 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,lineitem,orders,customer,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ o_year, sum(CASE WHEN nation = 'BRAZIL' diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql index 0e7e1a63262f7c3..cee9925fb5d9023 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,lineitem,partsupp,orders,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ nation, o_year, sum(amount) AS sum_profit diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql index 2a78f2931f6f7cc..c95a80fceef4fea 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) AS revenue, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql index eb0987288a78740..b23701e940722e6 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql @@ -1,5 +1,5 @@ -- tables: partsupp,supplier,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ ps_partkey, sum(ps_supplycost * ps_availqty) AS value FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql index d72e552a3396575..e8893e71e4eae31 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql @@ -1,5 +1,5 @@ -- tables: orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql index d222de86947098f..9db2da60ee43883 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql @@ -1,5 +1,5 @@ -- tables: customer -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_count, count(*) AS custdist FROM ( diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql index b70a8abfd8a1fb1..70d7a57d07dcbc0 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ 100.00 * sum(CASE +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0 diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql index fd526df87b088c9..45f75ff985778b1 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql @@ -1,4 +1,4 @@ -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ s_suppkey, s_name, s_address, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql index c21c8fb494655e4..37a438c796129e8 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql @@ -1,5 +1,5 @@ -- tables: partsupp,part,supplier -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ p_brand, p_type, p_size, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql index ae11f9c699d236f..62f39a750c2c078 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ sum(l_extendedprice) / 7.0 AS avg_yearly +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, part diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql index 910b9ffc7f71ff8..2eb2505c01e75a2 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_name, c_custkey, o_orderkey, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql index b55400e4ae5fc80..16e543f87c0eda8 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ sum(l_extendedprice * (1 - l_discount)) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * (1 - l_discount)) AS revenue FROM lineitem, part diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql index 550c00b4e40d052..a2aca5679017521 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql @@ -1,5 +1,5 @@ -- tables: supplier,nation,partsupp,lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ s_name, s_address FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql index 099b5ad10657e34..7b4874f96c74317 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql @@ -1,5 +1,5 @@ -- tables: supplier,lineitem,orders,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ s_name, count(*) AS numwait FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql index fb0d71a6bc153c6..bf784175e0ed2d7 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql @@ -1,5 +1,5 @@ -- tables: orders,customer -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=false) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctbal