Merge branch 'master' into fix-projections

This commit is contained in:
Alexey Milovidov 2022-10-11 02:24:12 +03:00 committed by GitHub
commit 56e59dd536
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 57 additions and 8 deletions

View File

@ -1,7 +1,7 @@
# To run clang-tidy from CMake, build ClickHouse with -DENABLE_CLANG_TIDY=1. To show all warnings, it is
# recommended to pass "-k0" to Ninja.
# Enable all checks + disale selected checks. Feel free to remove disabled checks from below list if
# Enable all checks + disable selected checks. Feel free to remove disabled checks from below list if
# a) the new check is not controversial (this includes many checks in readability-* and google-*) or
# b) too noisy (checks with > 100 new warnings are considered noisy, this includes e.g. cppcoreguidelines-*).

View File

@ -1,7 +1,7 @@
#if defined(OS_LINUX)
# include <sys/syscall.h>
#endif
#include <stdlib.h>
#include <cstdlib>
#include <unistd.h>
#include <base/safeExit.h>
#include <base/defines.h>

View File

@ -80,6 +80,7 @@ set(LLVM_ENABLE_LIBXML2 0 CACHE INTERNAL "")
set(LLVM_ENABLE_LIBEDIT 0 CACHE INTERNAL "")
set(LLVM_ENABLE_LIBPFM 0 CACHE INTERNAL "")
set(LLVM_ENABLE_ZLIB 0 CACHE INTERNAL "")
set(LLVM_ENABLE_ZSTD 0 CACHE INTERNAL "")
set(LLVM_ENABLE_Z3_SOLVER 0 CACHE INTERNAL "")
set(LLVM_INCLUDE_TOOLS 0 CACHE INTERNAL "")
set(LLVM_BUILD_TOOLS 0 CACHE INTERNAL "")

View File

@ -120,7 +120,11 @@ use_cron()
if [ -x "/bin/systemctl" ] && [ -f /etc/systemd/system/clickhouse-server.service ] && [ -d /run/systemd/system ]; then
return 1
fi
# 2. disabled by config
# 2. checking whether the config is existed
if [ ! -f "$CLICKHOUSE_CRONFILE" ]; then
return 1
fi
# 3. disabled by config
if [ -z "$CLICKHOUSE_CRONFILE" ]; then
return 2
fi

View File

@ -114,4 +114,6 @@ struct AggregateUtils
}
};
const String & getAggregateFunctionCanonicalNameIfAny(const String & name);
}

View File

@ -42,7 +42,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
*/
#define COMMON_SETTINGS(M) \
M(Dialect, dialect, Dialect::clickhouse, "Which SQL dialect will be used to parse query", 0)\
M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \

View File

@ -88,4 +88,6 @@ private:
String getFactoryName() const override { return "FunctionFactory"; }
};
const String & getFunctionCanonicalNameIfAny(const String & name);
}

View File

@ -4,12 +4,12 @@
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Functions/FunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
namespace DB
{
const String & getFunctionCanonicalNameIfAny(const String & name);
const String & getAggregateFunctionCanonicalNameIfAny(const String & name);
void FunctionNameNormalizer::visit(IAST * ast)
{
if (!ast)

View File

@ -44,6 +44,12 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
if (!read_from_merge_tree)
return 0;
/// if reading from merge tree doesn't provide any output order, we can do nothing
/// it means that no ordering can provided or supported for a particular sorting key
/// for example, tuple() or sipHash(string)
if (read_from_merge_tree->getOutputStream().sort_description.empty())
return 0;
/// find non-const columns in DISTINCT
const ColumnsWithTypeAndName & distinct_columns = pre_distinct->getOutputStream().header.getColumnsWithTypeAndName();
std::set<std::string_view> non_const_columns;

View File

@ -954,6 +954,8 @@ void MergeTreeData::loadDataPartsFromDisk(
/// Parallel loading of data parts.
pool.setMaxThreads(std::min(static_cast<size_t>(settings->max_part_loading_threads), num_parts));
size_t num_threads = pool.getMaxThreads();
LOG_DEBUG(log, "Going to use {} threads to load parts", num_threads);
std::vector<size_t> parts_per_thread(num_threads, num_parts / num_threads);
for (size_t i = 0ul; i < num_parts % num_threads; ++i)
++parts_per_thread[i];
@ -1016,6 +1018,8 @@ void MergeTreeData::loadDataPartsFromDisk(
auto part_opt = MergeTreePartInfo::tryParsePartName(part_name, format_version);
if (!part_opt)
return;
LOG_TRACE(log, "Loading part {} from disk {}", part_name, part_disk_ptr->getName());
const auto & part_info = *part_opt;
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, part_disk_ptr, 0);
auto data_part_storage = std::make_shared<DataPartStorageOnDisk>(single_disk_volume, relative_data_path, part_name);
@ -1119,6 +1123,7 @@ void MergeTreeData::loadDataPartsFromDisk(
}
addPartContributionToDataVolume(part);
LOG_TRACE(log, "Finished part {} load on disk {}", part_name, part_disk_ptr->getName());
};
std::mutex part_select_mutex;
@ -1311,8 +1316,10 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
size_t num_parts = 0;
std::queue<std::vector<std::pair<String, DiskPtr>>> parts_queue;
for (auto & [_, disk_parts] : disk_part_map)
for (auto & [disk_name, disk_parts] : disk_part_map)
{
LOG_INFO(log, "Found {} parts for disk '{}' to load", disk_parts.size(), disk_name);
if (disk_parts.empty())
continue;
num_parts += disk_parts.size();

View File

@ -110,3 +110,10 @@ select distinct a, b, x, y from (select a, b, 1 as x, 2 as y from distinct_in_or
0
-- check that distinct in order WITHOUT order by and WITH filter returns the same result as ordinary distinct
0
-- bug 42185, distinct in order and empty sort description
-- distinct in order, sorting key tuple()
1
0
-- distinct in order, sorting key contains function
2000-01-01 00:00:00
2000-01-01

View File

@ -95,3 +95,23 @@ select count() as diff from (select distinct * from distinct_in_order except sel
drop table if exists distinct_in_order;
drop table if exists ordinary_distinct;
drop table if exists distinct_cardinality_low;
-- bug 42185
drop table if exists sorting_key_empty_tuple;
drop table if exists sorting_key_contain_function;
select '-- bug 42185, distinct in order and empty sort description';
select '-- distinct in order, sorting key tuple()';
create table sorting_key_empty_tuple (a int, b int) engine=MergeTree() order by tuple();
insert into sorting_key_empty_tuple select number % 2, number % 5 from numbers(1,10);
select distinct a from sorting_key_empty_tuple;
select '-- distinct in order, sorting key contains function';
create table sorting_key_contain_function (datetime DateTime, a int) engine=MergeTree() order by (toDate(datetime));
insert into sorting_key_contain_function values ('2000-01-01', 1);
insert into sorting_key_contain_function values ('2000-01-01', 2);
select distinct datetime from sorting_key_contain_function;
select distinct toDate(datetime) from sorting_key_contain_function;
drop table sorting_key_empty_tuple;
drop table sorting_key_contain_function;