mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge remote-tracking branch 'rschu1ze/master' into fix-64136
This commit is contained in:
commit
96ebaa17d3
@ -68,8 +68,9 @@ if (ENABLE_CHECK_HEAVY_BUILDS)
|
||||
set (RLIMIT_AS 20000000000)
|
||||
endif()
|
||||
|
||||
# For some files currently building RISCV64 might be too slow. TODO: Improve compilation times per file
|
||||
if (ARCH_RISCV64)
|
||||
# For some files currently building RISCV64/LOONGARCH64 might be too slow.
|
||||
# TODO: Improve compilation times per file
|
||||
if (ARCH_RISCV64 OR ARCH_LOONGARCH64)
|
||||
set (RLIMIT_CPU 1800)
|
||||
endif()
|
||||
|
||||
|
@ -14,7 +14,7 @@ struct Settings;
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
||||
extern const int CORRUPTED_DATA;
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
@ -198,7 +198,7 @@ public:
|
||||
this->data(place).value().read(buf, *serialization_val, arena);
|
||||
if (unlikely(this->data(place).value().has() != this->data(place).result().has()))
|
||||
throw Exception(
|
||||
ErrorCodes::CORRUPTED_DATA,
|
||||
ErrorCodes::INCORRECT_DATA,
|
||||
"Invalid state of the aggregate function {}: has_value ({}) != has_result ({})",
|
||||
getName(),
|
||||
this->data(place).value().has(),
|
||||
|
@ -42,7 +42,7 @@ private:
|
||||
return;
|
||||
|
||||
const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage();
|
||||
bool is_final_supported = storage && storage->supportsFinal();
|
||||
bool is_final_supported = storage && !storage->isRemote() && storage->supportsFinal();
|
||||
if (!is_final_supported)
|
||||
return;
|
||||
|
||||
|
@ -192,7 +192,7 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node)
|
||||
void QueryTreePassManager::runOnlyResolve(QueryTreeNodePtr query_tree_node)
|
||||
{
|
||||
// Run only QueryAnalysisPass and GroupingFunctionsResolvePass passes.
|
||||
run(query_tree_node, 2);
|
||||
run(query_tree_node, 3);
|
||||
}
|
||||
|
||||
void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index)
|
||||
@ -249,6 +249,7 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze)
|
||||
{
|
||||
manager.addPass(std::make_unique<QueryAnalysisPass>(only_analyze));
|
||||
manager.addPass(std::make_unique<GroupingFunctionsResolvePass>());
|
||||
manager.addPass(std::make_unique<AutoFinalOnQueryPass>());
|
||||
|
||||
manager.addPass(std::make_unique<RemoveUnusedProjectionColumnsPass>());
|
||||
manager.addPass(std::make_unique<FunctionToSubcolumnsPass>());
|
||||
@ -294,7 +295,6 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze)
|
||||
|
||||
manager.addPass(std::make_unique<LogicalExpressionOptimizerPass>());
|
||||
|
||||
manager.addPass(std::make_unique<AutoFinalOnQueryPass>());
|
||||
manager.addPass(std::make_unique<CrossToInnerJoinPass>());
|
||||
manager.addPass(std::make_unique<ShardNumColumnToFunctionPass>());
|
||||
|
||||
|
@ -721,11 +721,10 @@ public:
|
||||
if (!block.checkCheckSum())
|
||||
{
|
||||
std::string calculated_check_sum = std::to_string(block.calculateCheckSum());
|
||||
std::string check_sum = std::to_string(block.getCheckSum());
|
||||
std::string expected_check_sum = std::to_string(block.getCheckSum());
|
||||
throw Exception(ErrorCodes::CORRUPTED_DATA,
|
||||
"Cache data corrupted. Checksum validation failed. Calculated {} in block {}",
|
||||
calculated_check_sum,
|
||||
check_sum);
|
||||
"Cache data corrupted. Checksum validation failed. Calculated {} expected in block {}, in file {}",
|
||||
calculated_check_sum, expected_check_sum, file_path);
|
||||
}
|
||||
|
||||
func(blocks_to_fetch[block_to_fetch_index], block.getBlockData());
|
||||
|
@ -31,7 +31,7 @@ extract_into_parent_list(clickhouse_functions_headers dbms_headers
|
||||
add_library(clickhouse_functions_obj OBJECT ${clickhouse_functions_headers} ${clickhouse_functions_sources})
|
||||
if (OMIT_HEAVY_DEBUG_SYMBOLS)
|
||||
target_compile_options(clickhouse_functions_obj PRIVATE "-g0")
|
||||
set_source_files_properties(${DBMS_FUNCTIONS} PROPERTIES COMPILE_FLAGS "-g0")
|
||||
set_source_files_properties(${DBMS_FUNCTIONS} DIRECTORY .. PROPERTIES COMPILE_FLAGS "-g0")
|
||||
endif()
|
||||
|
||||
list (APPEND OBJECT_LIBS $<TARGET_OBJECTS:clickhouse_functions_obj>)
|
||||
|
@ -1392,7 +1392,16 @@ void executeQuery(
|
||||
const char * begin;
|
||||
const char * end;
|
||||
|
||||
istr.nextIfAtEnd();
|
||||
try
|
||||
{
|
||||
istr.nextIfAtEnd();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If buffer contains invalid data and we failed to decompress, we still want to have some information about the query in the log.
|
||||
logQuery("<cannot parse>", context, /* internal = */ false, QueryProcessingStage::Complete);
|
||||
throw;
|
||||
}
|
||||
|
||||
size_t max_query_size = context->getSettingsRef().max_query_size;
|
||||
|
||||
|
@ -707,11 +707,11 @@ void HTTPHandler::processQuery(
|
||||
/// The data can also be compressed using incompatible internal algorithm. This is indicated by
|
||||
/// 'decompress' query parameter.
|
||||
std::unique_ptr<ReadBuffer> in_post_maybe_compressed;
|
||||
bool in_post_compressed = false;
|
||||
bool is_in_post_compressed = false;
|
||||
if (params.getParsed<bool>("decompress", false))
|
||||
{
|
||||
in_post_maybe_compressed = std::make_unique<CompressedReadBuffer>(*in_post);
|
||||
in_post_compressed = true;
|
||||
in_post_maybe_compressed = std::make_unique<CompressedReadBuffer>(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true);
|
||||
is_in_post_compressed = true;
|
||||
}
|
||||
else
|
||||
in_post_maybe_compressed = std::move(in_post);
|
||||
@ -845,7 +845,7 @@ void HTTPHandler::processQuery(
|
||||
|
||||
/// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on,
|
||||
/// checksums of client data compressed with internal algorithm are not checked.
|
||||
if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress)
|
||||
if (is_in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress)
|
||||
static_cast<CompressedReadBuffer &>(*in_post_maybe_compressed).disableChecksumming();
|
||||
|
||||
/// Add CORS header if 'add_http_cors_header' setting is turned on send * in Access-Control-Allow-Origin
|
||||
|
@ -132,3 +132,7 @@ SELECT * FROM merge_table ORDER BY id, val;
|
||||
2 a
|
||||
2 b
|
||||
3 c
|
||||
select sum(number) from numbers(10) settings final=1;
|
||||
45
|
||||
select sum(number) from remote('127.0.0.{1,2}', numbers(10)) settings final=1;
|
||||
90
|
||||
|
@ -102,3 +102,6 @@ insert into table_to_merge_c values (3,'c');
|
||||
-- expected output:
|
||||
-- 1 c, 2 a, 2 b, 3 c
|
||||
SELECT * FROM merge_table ORDER BY id, val;
|
||||
|
||||
select sum(number) from numbers(10) settings final=1;
|
||||
select sum(number) from remote('127.0.0.{1,2}', numbers(10)) settings final=1;
|
||||
|
@ -103,11 +103,11 @@ SELECT '2^30-1', maxMerge(x) from (select CAST(unhex('ffffff3f') || randomString
|
||||
SELECT '1M without 0', length(maxMerge(x)) from (select CAST(unhex('00001000') || randomString(0x00100000 - 1) || 'x', 'AggregateFunction(max, String)') as x);
|
||||
SELECT '1M with 0', length(maxMerge(x)) from (select CAST(unhex('00001000') || randomString(0x00100000 - 1) || '\0', 'AggregateFunction(max, String)') as x);
|
||||
|
||||
SELECT 'fuzz1', finalizeAggregation(CAST(unhex('3000000\0303132333435363738393031323334353637383930313233343536373839303132333435363738393031323334353600010000000000000000'), 'AggregateFunction(argMax, String, UInt64)')); -- { serverError CORRUPTED_DATA }
|
||||
SELECT 'fuzz1', finalizeAggregation(CAST(unhex('3000000\0303132333435363738393031323334353637383930313233343536373839303132333435363738393031323334353600010000000000000000'), 'AggregateFunction(argMax, String, UInt64)')); -- { serverError INCORRECT_DATA }
|
||||
SELECT 'fuzz2', finalizeAggregation(CAST(unhex('04000000' || '30313233' || '01' || 'ffffffffffffffff'), 'AggregateFunction(argMax, String, UInt64)')) as x, length(x);
|
||||
SELECT 'fuzz3', finalizeAggregation(CAST(unhex('04000000' || '30313233' || '00' || 'ffffffffffffffff'), 'AggregateFunction(argMax, String, UInt64)')) as x, length(x); -- { serverError CORRUPTED_DATA }
|
||||
SELECT 'fuzz4', finalizeAggregation(CAST(unhex('04000000' || '30313233' || '00'), 'AggregateFunction(argMax, String, UInt64)')) as x, length(x); -- { serverError CORRUPTED_DATA }
|
||||
SELECT 'fuzz5', finalizeAggregation(CAST(unhex('0100000000000000000FFFFFFFF0'), 'AggregateFunction(argMax, UInt64, String)')); -- { serverError CORRUPTED_DATA }
|
||||
SELECT 'fuzz3', finalizeAggregation(CAST(unhex('04000000' || '30313233' || '00' || 'ffffffffffffffff'), 'AggregateFunction(argMax, String, UInt64)')) as x, length(x); -- { serverError INCORRECT_DATA }
|
||||
SELECT 'fuzz4', finalizeAggregation(CAST(unhex('04000000' || '30313233' || '00'), 'AggregateFunction(argMax, String, UInt64)')) as x, length(x); -- { serverError INCORRECT_DATA }
|
||||
SELECT 'fuzz5', finalizeAggregation(CAST(unhex('0100000000000000000FFFFFFFF0'), 'AggregateFunction(argMax, UInt64, String)')); -- { serverError INCORRECT_DATA }
|
||||
|
||||
|
||||
drop table if exists aggr;
|
||||
|
@ -7,8 +7,6 @@ export LC_ALL=C # The "total" should be printed without localization
|
||||
TU_EXCLUDES=(
|
||||
AggregateFunctionUniq
|
||||
Aggregator
|
||||
# FIXME: Exclude for now
|
||||
FunctionsConversion
|
||||
)
|
||||
|
||||
if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 50000000) print }' \
|
||||
|
Loading…
Reference in New Issue
Block a user