Merge branch 'master' into tacit

This commit is contained in:
Alexey Milovidov 2024-05-05 00:20:41 +02:00
commit 77c0ebd3ee
93 changed files with 623 additions and 253 deletions

View File

@ -34,19 +34,25 @@ curl https://clickhouse.com/ | sh
Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know.
* [v24.3 Community Call](https://clickhouse.com/company/events/v24-3-community-release-call) - Mar 26
* [v24.4 Community Call](https://clickhouse.com/company/events/v24-4-community-release-call) - Apr 30
* [v24.5 Community Call](https://clickhouse.com/company/events/v24-5-community-release-call) - May 30
## Upcoming Events
Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `<at>` clickhouse `<dot>` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc.
* [ClickHouse Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28
* [ClickHouse Meetup in Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/300405581/) - May 4
* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angeles](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22
* [ClickHouse & Confluent Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28
* [ClickHouse Meetup in Stockholm](https://www.meetup.com/clickhouse-stockholm-user-group/events/299752651/) - Jun 3
* [ClickHouse Meetup @ Cloudflare - San Francisco](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/300523061/) - Jun 4
* [ClickHouse (クリックハウス) Meetup Tokyo](https://www.meetup.com/clickhouse-tokyo-user-group/events/300798053/) - Jun 5
* [ClickHouse Meetup in Amsterdam](https://www.meetup.com/clickhouse-netherlands-user-group/events/300781068/) - Jun 27
* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9
* [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"
* **Recording available**: [**v24.2 Release Call**](https://www.youtube.com/watch?v=iN2y-TK8f3A) All the features of 24.2, one convenient video! Watch it now!
* **Recording available**: [**v24.4 Release Call**](https://www.youtube.com/watch?v=dtUqgcfOGmE) All the features of 24.4, one convenient video! Watch it now!
## Interested in joining ClickHouse and making it your full-time job?

View File

@ -8,6 +8,11 @@ set (DEFAULT_LIBS "-nodefaultlibs")
if (COMPILER_CLANG)
execute_process (COMMAND ${CMAKE_CXX_COMPILER} --target=${CMAKE_CXX_COMPILER_TARGET} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE)
# Apparently, in clang-19, the UBSan support library for C++ was moved out into ubsan_standalone_cxx.a, so we have to include both.
if (SANITIZE STREQUAL undefined)
string(REPLACE "builtins.a" "ubsan_standalone_cxx.a" EXTRA_BUILTINS_LIBRARY "${BUILTINS_LIBRARY}")
endif ()
if (NOT EXISTS "${BUILTINS_LIBRARY}")
set (BUILTINS_LIBRARY "-lgcc")
endif ()
@ -15,11 +20,11 @@ endif ()
if (OS_ANDROID)
# pthread and rt are included in libc
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl")
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${EXTRA_BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl")
elseif (USE_MUSL)
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -static -lc")
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${EXTRA_BUILTINS_LIBRARY} ${COVERAGE_OPTION} -static -lc")
else ()
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl")
set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${EXTRA_BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl")
endif ()
message(STATUS "Default libraries: ${DEFAULT_LIBS}")

View File

@ -78,7 +78,8 @@ function configure()
randomize_config_boolean_value use_compression zookeeper
fi
randomize_config_boolean_value allow_experimental_block_number_column block_number
randomize_config_boolean_value enable_block_number_column block_number
randomize_config_boolean_value enable_block_offset_column block_number
# for clickhouse-server (via service)
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment

View File

@ -0,0 +1,14 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.8.14.6-lts (967e51c1d6b) FIXME as compared to v23.8.13.25-lts (37e034f903e)
#### Bug Fix (user-visible misbehavior in an official stable release)
* Set server name for SSL handshake in MongoDB engine [#63122](https://github.com/ClickHouse/ClickHouse/pull/63122) ([Alexander Gololobov](https://github.com/davenger)).
* Use user specified db instead of "config" for MongoDB wire protocol version check [#63126](https://github.com/ClickHouse/ClickHouse/pull/63126) ([Alexander Gololobov](https://github.com/davenger)).

View File

@ -1024,6 +1024,7 @@ The result of the function depends on the affected data blocks and the order of
:::note
Only returns neighbor inside the currently processed data block.
Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead.
:::
The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user.
@ -1134,6 +1135,7 @@ Returns 0 for the first row, and for subsequent rows the difference to the previ
:::note
Only returns differences inside the currently processed data block.
Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead.
:::
The result of the function depends on the affected data blocks and the order of data in the block.
@ -1207,6 +1209,10 @@ WHERE diff != 1
## runningDifferenceStartingWithFirstValue
:::note
This function is DEPRECATED (see the note for `runningDifference`).
:::
Same as [runningDifference](./other-functions.md#other_functions-runningdifference), but returns the value of the first row as the value on the first row.
## runningConcurrency
@ -1930,6 +1936,7 @@ Accumulates the states of an aggregate function for each row of a data block.
:::note
The state is reset for each new block of data.
Because of this error-prone behavior the function is DEPRECATED, please use proper window functions instead.
:::
**Syntax**

View File

@ -7,6 +7,8 @@
#include <random>
#include <string_view>
#include <pcg_random.hpp>
#include <Poco/UUID.h>
#include <Poco/UUIDGenerator.h>
#include <Poco/Util/Application.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
@ -80,6 +82,7 @@ public:
double max_time_,
size_t confidence_,
const String & query_id_,
const String & query_id_prefix_,
const String & query_to_execute_,
size_t max_consecutive_errors_,
bool continue_on_errors_,
@ -98,6 +101,7 @@ public:
max_time(max_time_),
confidence(confidence_),
query_id(query_id_),
query_id_prefix(query_id_prefix_),
query_to_execute(query_to_execute_),
continue_on_errors(continue_on_errors_),
max_consecutive_errors(max_consecutive_errors_),
@ -205,6 +209,7 @@ private:
double max_time;
size_t confidence;
String query_id;
String query_id_prefix;
String query_to_execute;
bool continue_on_errors;
size_t max_consecutive_errors;
@ -463,8 +468,11 @@ private:
RemoteQueryExecutor executor(
*entry, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage);
if (!query_id.empty())
executor.setQueryId(query_id);
else if (!query_id_prefix.empty())
executor.setQueryId(query_id_prefix + "_" + Poco::UUIDGenerator().createRandom().toString());
Progress progress;
executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });
@ -617,6 +625,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
("stacktrace", "print stack traces of exceptions")
("confidence", value<size_t>()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
("query_id", value<std::string>()->default_value(""), "")
("query_id_prefix", value<std::string>()->default_value(""), "")
("max-consecutive-errors", value<size_t>()->default_value(0), "set number of allowed consecutive errors")
("ignore-error,continue_on_errors", "continue testing even if a query fails")
("reconnect", "establish new connection for every query")
@ -671,6 +680,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
options["timelimit"].as<double>(),
options["confidence"].as<size_t>(),
options["query_id"].as<std::string>(),
options["query_id_prefix"].as<std::string>(),
options["query"].as<std::string>(),
options["max-consecutive-errors"].as<size_t>(),
options.count("ignore-error"),

View File

@ -84,7 +84,7 @@ private:
for (const auto & file_name : file_names)
{
auto path = relative_path + "/" + file_name;
auto path = relative_path.empty() ? file_name : (relative_path + "/" + file_name);
if (disk->isDirectory(path))
listRecursive(disk, path);
}

View File

@ -97,7 +97,8 @@ void DisksApp::processOptions()
DisksApp::~DisksApp()
{
global_context->shutdown();
if (global_context)
global_context->shutdown();
}
void DisksApp::init(std::vector<String> & common_arguments)

View File

@ -3,6 +3,7 @@
#include <sys/resource.h>
#include <Common/logger_useful.h>
#include <Common/formatReadable.h>
#include <Core/UUID.h>
#include <base/getMemoryAmount.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/String.h>
@ -35,7 +36,6 @@
#include <Loggers/OwnPatternFormatter.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/UseSSL.h>
#include <IO/SharedThreadPools.h>
#include <Parsers/ASTInsertQuery.h>
@ -48,7 +48,6 @@
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
#include <Formats/FormatFactory.h>
#include <boost/algorithm/string/replace.hpp>
#include <boost/program_options/options_description.hpp>
#include <base/argsToConfig.h>
@ -211,7 +210,7 @@ void LocalServer::tryInitPath()
else
{
// The path is not provided explicitly - use a unique path in the system temporary directory
// (or in the current dir if temporary don't exist)
// (or in the current dir if a temporary doesn't exist)
LoggerRawPtr log = &logger();
std::filesystem::path parent_folder;
std::filesystem::path default_path;
@ -237,39 +236,30 @@ void LocalServer::tryInitPath()
/// as we can't accurately distinguish those situations we don't touch any existent folders
/// we just try to pick some free name for our working folder
default_path = parent_folder / fmt::format("clickhouse-local-{}-{}-{}", getpid(), time(nullptr), randomSeed());
default_path = parent_folder / fmt::format("clickhouse-local-{}", UUIDHelpers::generateV4());
if (exists(default_path))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to create working directory: {} already exists.", default_path.string());
if (fs::exists(default_path))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to set up the working directory: {} already exists.", default_path.string());
create_directory(default_path);
/// The directory can be created lazily during the runtime.
temporary_directory_to_delete = default_path;
path = default_path.string();
LOG_DEBUG(log, "Working directory created: {}", path);
}
if (path.back() != '/')
path += '/';
global_context->setPath(fs::path(path) / "");
fs::create_directories(fs::path(path) / "user_defined/");
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
fs::create_directories(fs::path(path) / "metadata_dropped/");
global_context->setTemporaryStoragePath(fs::path(path) / "tmp" / "", 0);
global_context->setFlagsPath(fs::path(path) / "flags" / "");
global_context->setPath(path);
global_context->setTemporaryStoragePath(path + "tmp/", 0);
global_context->setFlagsPath(path + "flags");
global_context->setUserFilesPath(""); // user's files are everywhere
global_context->setUserFilesPath(""); /// user's files are everywhere
std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/");
global_context->setUserScriptsPath(user_scripts_path);
fs::create_directories(user_scripts_path);
/// top_level_domains_lists
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/");
const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
if (!top_level_domains_path.empty())
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
@ -299,10 +289,9 @@ void LocalServer::cleanup()
// Delete the temporary directory if needed.
if (temporary_directory_to_delete)
{
const auto dir = *temporary_directory_to_delete;
LOG_DEBUG(&logger(), "Removing temporary directory: {}", temporary_directory_to_delete->string());
fs::remove_all(*temporary_directory_to_delete);
temporary_directory_to_delete.reset();
LOG_DEBUG(&logger(), "Removing temporary directory: {}", dir.string());
remove_all(dir);
}
}
catch (...)
@ -481,6 +470,9 @@ try
registerFormats();
processConfig();
SCOPE_EXIT({ cleanup(); });
initTTYBuffer(toProgressOption(config().getString("progress", "default")));
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
@ -530,16 +522,12 @@ try
}
catch (const DB::Exception & e)
{
cleanup();
bool need_print_stack_trace = config().getBool("stacktrace", false);
std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl;
return e.code() ? e.code() : -1;
}
catch (...)
{
cleanup();
std::cerr << getCurrentExceptionMessage(false) << std::endl;
return getCurrentExceptionCode();
}
@ -705,6 +693,7 @@ void LocalServer::processConfig()
if (config().has("path"))
{
String path = global_context->getPath();
fs::create_directories(fs::path(path));
/// Lock path directory before read
status.emplace(fs::path(path) / "status", StatusFile::write_full_info);

View File

@ -1205,11 +1205,11 @@ try
}
{
fs::create_directories(path / "data/");
fs::create_directories(path / "metadata/");
fs::create_directories(path / "data");
fs::create_directories(path / "metadata");
/// Directory with metadata of tables, which was marked as dropped by Atomic database
fs::create_directories(path / "metadata_dropped/");
fs::create_directories(path / "metadata_dropped");
}
if (config().has("interserver_http_port") && config().has("interserver_https_port"))

View File

@ -47,6 +47,11 @@ fn skim_impl(prefix: &CxxString, words: &CxxVector<CxxString>) -> Result<String,
.height(Some("30%"))
.query(Some(prefix.to_str().unwrap()))
.tac(true)
// Do not clear on start and clear on exit will clear skim output from the terminal.
//
// Refs: https://github.com/lotabout/skim/issues/494#issuecomment-1776565846
.no_clear_start(true)
.no_clear(false)
.tiebreak(Some("-score".to_string()))
// Exact mode performs better for SQL.
//

View File

@ -63,6 +63,11 @@ struct AggregateFunctionBoundingRatioData
{
*this = other;
}
else if (other.empty)
{
// if other.empty = true, other.x/other.y may be uninitialized values,
// so don't use them to update this->state
}
else
{
if (other.left.x < left.x)

View File

@ -5725,6 +5725,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
checkFunctionNodeHasEmptyNullsAction(function_node);
const auto & untuple_argument = function_arguments[0];
/// Handle this special case first as `getResultType()` might return nullptr
if (untuple_argument->as<LambdaNode>())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function untuple can't have lambda-expressions as arguments");
auto result_type = untuple_argument->getResultType();
const auto * tuple_data_type = typeid_cast<const DataTypeTuple *>(result_type.get());
if (!tuple_data_type)

View File

@ -72,6 +72,12 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory)
configuration.container = config.getString(config_prefix + ".container");
configuration.account_name = config.getString(config_prefix + ".account_name");
configuration.account_key = config.getString(config_prefix + ".account_key");
if (config.has(config_prefix + ".account_name") && config.has(config_prefix + ".account_key"))
{
configuration.account_name = config.getString(config_prefix + ".account_name");
configuration.account_key = config.getString(config_prefix + ".account_key");
}
}
if (args.size() > 1)

View File

@ -599,6 +599,7 @@
M(718, TOO_SLOW_PARSING) \
M(719, QUERY_CACHE_USED_WITH_SYSTEM_TABLE) \
M(720, USER_EXPIRED) \
M(721, DEPRECATED_FUNCTION) \
\
M(900, DISTRIBUTED_CACHE_ERROR) \
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \

View File

@ -113,17 +113,17 @@ private:
public:
explicit LoadFromSQL(ContextPtr context_)
: WithContext(context_)
, metadata_path(
fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY)
, metadata_path(fs::weakly_canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY)
{
if (fs::exists(metadata_path))
cleanUp();
else
fs::create_directories(metadata_path);
cleanup();
}
std::vector<std::string> listCollections() const
{
if (!fs::exists(metadata_path))
return {};
std::vector<std::string> collection_names;
fs::directory_iterator it{metadata_path};
for (; it != fs::directory_iterator{}; ++it)
@ -280,7 +280,7 @@ private:
/// Delete .tmp files. They could be left undeleted in case of
/// some exception or abrupt server restart.
void cleanUp()
void cleanup()
{
fs::directory_iterator it{metadata_path};
std::vector<std::string> files_to_remove;
@ -308,11 +308,11 @@ private:
return create_query;
}
static void writeCreateQueryToMetadata(
void writeCreateQueryToMetadata(
const ASTCreateNamedCollectionQuery & query,
const std::string & path,
const Settings & settings,
bool replace = false)
bool replace = false) const
{
if (!replace && fs::exists(path))
{
@ -322,6 +322,8 @@ private:
path);
}
fs::create_directories(metadata_path);
auto tmp_path = path + ".tmp";
String formatted_query = serializeAST(query);
WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL);

View File

@ -42,14 +42,23 @@ namespace ErrorCodes
extern const int CANNOT_CREATE_FILE;
}
struct statvfs getStatVFS(const String & path)
struct statvfs getStatVFS(String path)
{
struct statvfs fs;
while (statvfs(path.c_str(), &fs) != 0)
{
if (errno == EINTR)
continue;
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)");
/// Sometimes we create directories lazily, so we can request free space in a directory that yet to be created.
auto fs_path = std::filesystem::path(path);
if (errno == ENOENT && fs_path.has_parent_path())
{
path = fs_path.parent_path();
continue;
}
ErrnoException::throwFromPath(ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)");
}
return fs;
}

View File

@ -56,7 +56,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path);
#endif
String getFilesystemName([[maybe_unused]] const String & mount_point);
struct statvfs getStatVFS(const String & path);
struct statvfs getStatVFS(String path);
/// Returns true if path starts with prefix path
bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path);

View File

@ -912,6 +912,7 @@ class IColumn;
M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \
M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \
M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \
M(Bool, allow_deprecated_functions, false, "Allow usage of deprecated functions", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS.
@ -1122,7 +1123,7 @@ class IColumn;
M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \
M(ParquetCompression, output_format_parquet_compression_method, "zstd", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \
M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \
M(Bool, output_format_parquet_use_custom_encoder, false, "Use a faster Parquet encoder implementation.", 0) \
M(Bool, output_format_parquet_use_custom_encoder, true, "Use a faster Parquet encoder implementation.", 0) \
M(Bool, output_format_parquet_parallel_encoding, true, "Do Parquet encoding in multiple threads. Requires output_format_parquet_use_custom_encoder.", 0) \
M(UInt64, output_format_parquet_data_page_size, 1024 * 1024, "Target page size in bytes, before compression.", 0) \
M(UInt64, output_format_parquet_batch_size, 1024, "Check page size every this many rows. Consider decreasing if you have columns with average values size above a few KBs.", 0) \

View File

@ -85,6 +85,10 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
{"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."},
}},
{"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"},
{"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"},
{"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"},
@ -96,7 +100,6 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"},
{"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"},
{"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
{"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"},
{"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"},
}},

View File

@ -137,6 +137,9 @@ void DatabaseAtomic::dropTableImpl(ContextPtr local_context, const String & tabl
std::lock_guard lock(mutex);
table = getTableUnlocked(table_name);
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
fs::create_directory(fs::path(table_metadata_path_drop).parent_path());
auto txn = local_context->getZooKeeperMetadataTransaction();
if (txn && !local_context->isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database

View File

@ -935,6 +935,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
query_context->setSetting("allow_experimental_window_functions", 1);
query_context->setSetting("allow_experimental_geo_types", 1);
query_context->setSetting("allow_experimental_map_type", 1);
query_context->setSetting("allow_deprecated_functions", 1);
query_context->setSetting("allow_suspicious_low_cardinality_types", 1);
query_context->setSetting("allow_suspicious_fixed_string_types", 1);

View File

@ -36,8 +36,16 @@ CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
, update_queue(configuration.max_update_queue_size)
, update_pool(CurrentMetrics::CacheDictionaryThreads, CurrentMetrics::CacheDictionaryThreadsActive, CurrentMetrics::CacheDictionaryThreadsScheduled, configuration.max_threads_for_updates)
{
for (size_t i = 0; i < configuration.max_threads_for_updates; ++i)
update_pool.scheduleOrThrowOnError([this] { updateThreadFunction(); });
try
{
for (size_t i = 0; i < configuration.max_threads_for_updates; ++i)
update_pool.scheduleOrThrowOnError([this] { updateThreadFunction(); });
}
catch (...)
{
stopAndWait();
throw;
}
}
template <DictionaryKeyType dictionary_key_type>

View File

@ -18,7 +18,6 @@
#include <fcntl.h>
#include <sys/stat.h>
#include <Disks/DiskFactory.h>
#include <Disks/IO/WriteBufferFromTemporaryFile.h>
#include <Common/randomSeed.h>
@ -224,7 +223,7 @@ static UInt64 getTotalSpaceByName(const String & name, const String & disk_path,
{
struct statvfs fs;
if (name == "default") /// for default disk we get space from path/data/
fs = getStatVFS((fs::path(disk_path) / "data/").string());
fs = getStatVFS((fs::path(disk_path) / "data" / "").string());
else
fs = getStatVFS(disk_path);
UInt64 total_size = fs.f_blocks * fs.f_frsize;
@ -248,7 +247,7 @@ std::optional<UInt64> DiskLocal::getAvailableSpace() const
/// available for superuser only and for system purposes
struct statvfs fs;
if (name == "default") /// for default disk we get space from path/data/
fs = getStatVFS((fs::path(disk_path) / "data/").string());
fs = getStatVFS((fs::path(disk_path) / "data" / "").string());
else
fs = getStatVFS(disk_path);
UInt64 total_size = fs.f_bavail * fs.f_frsize;

View File

@ -21,6 +21,7 @@ public:
DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_,
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
DiskLocal(
const String & name_,
const String & path_,

View File

@ -5,6 +5,7 @@
#include <Common/Exception.h>
#include <Common/re2.h>
#include <azure/identity/managed_identity_credential.hpp>
#include <azure/identity/workload_identity_credential.hpp>
#include <azure/storage/blobs/blob_options.hpp>
#include <azure/core/http/curl_transport.hpp>
#include <Poco/Util/AbstractConfiguration.h>
@ -177,6 +178,12 @@ std::unique_ptr<T> getAzureBlobStorageClientWithAuth(
return std::make_unique<T>(url, storage_shared_key_credential, client_options);
}
if (config.getBool(config_prefix + ".use_workload_identity", false))
{
auto workload_identity_credential = std::make_shared<Azure::Identity::WorkloadIdentityCredential>();
return std::make_unique<T>(url, workload_identity_credential);
}
auto managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
return std::make_unique<T>(url, managed_identity_credential, client_options);
}

View File

@ -166,16 +166,14 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith
else
options.PageSizeHint = settings.get()->list_object_keys_size;
Azure::Storage::Blobs::ListBlobsPagedResponse blob_list_response;
while (true)
for (auto blob_list_response = client_ptr->ListBlobs(options); blob_list_response.HasPage(); blob_list_response.MoveToNextPage())
{
ProfileEvents::increment(ProfileEvents::AzureListObjects);
if (client_ptr->GetClickhouseOptions().IsClientForDisk)
ProfileEvents::increment(ProfileEvents::DiskAzureListObjects);
blob_list_response = client_ptr->ListBlobs(options);
auto blobs_list = blob_list_response.Blobs;
const auto & blobs_list = blob_list_response.Blobs;
for (const auto & blob : blobs_list)
{
@ -196,11 +194,6 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith
break;
options.PageSizeHint = keys_left;
}
if (blob_list_response.HasPage())
options.ContinuationToken = blob_list_response.NextPageToken;
else
break;
}
}

View File

@ -35,8 +35,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p
if (!disk)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk is not specified");
if (fs::path prefix_path(prefix); prefix_path.has_parent_path())
disk->createDirectories(prefix_path.parent_path());
disk->createDirectories((fs::path("") / prefix).parent_path());
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);

View File

@ -35,10 +35,12 @@ std::unique_ptr<IUserDefinedSQLObjectsStorage> createUserDefinedSQLObjectsStorag
}
return std::make_unique<UserDefinedSQLObjectsZooKeeperStorage>(global_context, config.getString(zookeeper_path_key));
}
String default_path = fs::path{global_context->getPath()} / "user_defined/";
String path = config.getString(disk_path_key, default_path);
return std::make_unique<UserDefinedSQLObjectsDiskStorage>(global_context, path);
else
{
String default_path = fs::path{global_context->getPath()} / "user_defined" / "";
String path = config.getString(disk_path_key, default_path);
return std::make_unique<UserDefinedSQLObjectsDiskStorage>(global_context, path);
}
}
}

View File

@ -5,6 +5,7 @@
namespace DB
{
class IUserDefinedSQLObjectsStorage;
std::unique_ptr<IUserDefinedSQLObjectsStorage> createUserDefinedSQLObjectsStorage(const ContextMutablePtr & global_context);

View File

@ -4,6 +4,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
namespace DB
@ -13,6 +14,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int DEPRECATED_FUNCTION;
}
namespace
@ -31,7 +33,18 @@ class FunctionNeighbor : public IFunction
{
public:
static constexpr auto name = "neighbor";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionNeighbor>(); }
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionNeighbor>();
}
/// Get the name of the function.
String getName() const override { return name; }

View File

@ -4,6 +4,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Common/AlignedBuffer.h>
#include <Common/Arena.h>
#include <Common/scope_guard_safe.h>
@ -16,6 +17,7 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int DEPRECATED_FUNCTION;
}
namespace
@ -34,8 +36,16 @@ class FunctionRunningAccumulate : public IFunction
{
public:
static constexpr auto name = "runningAccumulate";
static FunctionPtr create(ContextPtr)
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningAccumulate>();
}

View File

@ -1,16 +1,17 @@
#pragma once
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Common/assert_cast.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NumberTraits.h>
#include <DataTypes/DataTypeNullable.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Common/assert_cast.h>
namespace DB
@ -19,6 +20,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int DEPRECATED_FUNCTION;
}
@ -135,8 +137,15 @@ private:
public:
static constexpr auto name = FunctionRunningDifferenceName<is_first_line_zero>::name;
static FunctionPtr create(ContextPtr)
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningDifferenceImpl<is_first_line_zero>>();
}

View File

@ -1044,29 +1044,30 @@ try
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
fs::create_directories(path);
/// Clearing old temporary files.
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
if (fs::exists(path))
{
if (it->is_regular_file())
/// Clearing old temporary files.
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (startsWith(it->path().filename(), "tmp"))
if (it->is_regular_file())
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
if (startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string());
}
else
LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string());
/// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types.
}
/// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types.
}
}
catch (...)
{
DB::tryLogCurrentException(log, fmt::format(
"Caught exception while setup temporary path: {}. "
"Caught exception while setting up temporary path: {}. "
"It is ok to skip this exception as cleaning old temporary files is not necessary", path));
}
@ -1091,9 +1092,7 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size)
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock));
for (const auto & disk : volume->getDisks())
{
setupTmpPath(shared->log, disk->getPath());
}
TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
temporary_data_on_disk_settings.max_size_on_disk = max_size;

View File

@ -988,7 +988,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
/// we should load them and enqueue cleanup to remove data from store/ and metadata from ZooKeeper
std::map<String, StorageID> dropped_metadata;
String path = getContext()->getPath() + "metadata_dropped/";
String path = std::filesystem::path(getContext()->getPath()) / "metadata_dropped" / "";
if (!std::filesystem::exists(path))
{
@ -1043,10 +1043,11 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const
{
return getContext()->getPath() + "metadata_dropped/" +
escapeForFileName(table_id.getDatabaseName()) + "." +
escapeForFileName(table_id.getTableName()) + "." +
toString(table_id.uuid) + ".sql";
return std::filesystem::path(getContext()->getPath()) / "metadata_dropped" /
fmt::format("{}.{}.{}.sql",
escapeForFileName(table_id.getDatabaseName()),
escapeForFileName(table_id.getTableName()),
toString(table_id.uuid));
}
String DatabaseCatalog::getPathForMetadata(const StorageID & table_id) const

View File

@ -140,7 +140,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
/// Will write file with database metadata, if needed.
String database_name_escaped = escapeForFileName(database_name);
fs::path metadata_path = fs::canonical(getContext()->getPath());
fs::path metadata_path = fs::weakly_canonical(getContext()->getPath());
fs::create_directories(metadata_path / "metadata");
fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp");
fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql");

View File

@ -43,10 +43,10 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz
throw Exception(ErrorCodes::LOGICAL_ERROR, "Negative temporary data size");
}
size_t new_consumprion = stat.compressed_size + compressed_delta;
if (compressed_delta > 0 && settings.max_size_on_disk && new_consumprion > settings.max_size_on_disk)
size_t new_consumption = stat.compressed_size + compressed_delta;
if (compressed_delta > 0 && settings.max_size_on_disk && new_consumption > settings.max_size_on_disk)
throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES,
"Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, settings.max_size_on_disk);
"Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumption, settings.max_size_on_disk);
stat.compressed_size += compressed_delta;
stat.uncompressed_size += uncompressed_delta;

View File

@ -85,8 +85,6 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
bool group_by_use_nulls = planner_context->getQueryContext()->getSettingsRef().group_by_use_nulls &&
(query_node.isGroupByWithGroupingSets() || query_node.isGroupByWithRollup() || query_node.isGroupByWithCube());
bool is_secondary_query = planner_context->getQueryContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (query_node.hasGroupBy())
{
if (query_node.isGroupByWithGroupingSets())
@ -102,7 +100,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
auto is_constant_key = grouping_set_key_node->as<ConstantNode>() != nullptr;
group_by_with_constant_keys |= is_constant_key;
if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty())
if (is_constant_key && !aggregates_descriptions.empty())
continue;
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node);
@ -154,7 +152,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
auto is_constant_key = group_by_key_node->as<ConstantNode>() != nullptr;
group_by_with_constant_keys |= is_constant_key;
if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty())
if (is_constant_key && !aggregates_descriptions.empty())
continue;
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node);

View File

@ -33,7 +33,7 @@
/// * `def` and `rep` arrays can be longer than `primitive_column`, because they include nulls and
/// empty arrays; the values in primitive_column correspond to positions where def[i] == max_def.
///
/// If you do want to learn it, dremel paper: https://research.google/pubs/pub36632/
/// If you do want to learn it, see dremel paper: https://research.google/pubs/pub36632/
/// Instead of reading the whole paper, try staring at figures 2-3 for a while - it might be enough.
/// (Why does Parquet do all this instead of just storing array lengths and null masks? I'm not
/// really sure.)
@ -430,13 +430,16 @@ void prepareColumnNullable(
if (schemas[child_schema_idx].repetition_type == parq::FieldRepetitionType::REQUIRED)
{
/// Normal case: we just slap a FieldRepetitionType::OPTIONAL onto the nested column.
/// Normal case: the column inside Nullable is a primitive type (not Nullable/Array/Map).
/// Just slap a FieldRepetitionType::OPTIONAL onto it.
schemas[child_schema_idx].repetition_type = parq::FieldRepetitionType::OPTIONAL;
}
else
{
/// Weird case: Nullable(Nullable(...)). Or Nullable(Tuple(Nullable(...))), etc.
/// This is probably not allowed in ClickHouse, but let's support it just in case.
/// The nested column already has a nontrivial repetition type, so we have to wrap it in a
/// group and assign repetition type OPTIONAL to the group.
auto & schema = *schemas.insert(schemas.begin() + child_schema_idx, {});
schema.__set_repetition_type(parq::FieldRepetitionType::OPTIONAL);
schema.__set_name("nullable");

View File

@ -341,21 +341,21 @@ void MergeTreeDeduplicationLog::shutdown()
stopped = true;
if (current_writer)
{
current_writer->finalize();
try
{
current_writer->finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
current_writer.reset();
}
}
MergeTreeDeduplicationLog::~MergeTreeDeduplicationLog()
{
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
shutdown();
}
}

View File

@ -204,7 +204,7 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_)
throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString());
/// Ingest the SST file
static rocksdb::IngestExternalFileOptions ingest_options;
rocksdb::IngestExternalFileOptions ingest_options;
ingest_options.move_files = true; /// The temporary file is on the same disk, so move (or hardlink) file will be faster than copy
if (auto status = storage.rocksdb_ptr->IngestExternalFile({sst_file_path}, ingest_options); !status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString());

View File

@ -17,6 +17,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <azure/storage/common/storage_credential.hpp>
#include <azure/identity/workload_identity_credential.hpp>
#include <azure/identity/managed_identity_credential.hpp>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <Processors/Transforms/ExtractColumnsTransform.h>
@ -383,6 +384,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co
}
std::unique_ptr<BlobServiceClient> blob_service_client;
size_t pos = configuration.connection_url.find('?');
std::shared_ptr<Azure::Identity::ManagedIdentityCredential> managed_identity_credential;
if (storage_shared_key_credential)
{
@ -390,12 +392,20 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co
}
else
{
managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url, managed_identity_credential);
/// If conneciton_url does not have '?', then its not SAS
if (pos == std::string::npos)
{
auto workload_identity_credential = std::make_shared<Azure::Identity::WorkloadIdentityCredential>();
blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url, workload_identity_credential);
}
else
{
managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
blob_service_client = std::make_unique<BlobServiceClient>(configuration.connection_url, managed_identity_credential);
}
}
std::string final_url;
size_t pos = configuration.connection_url.find('?');
if (pos != std::string::npos)
{
auto url_without_sas = configuration.connection_url.substr(0, pos);
@ -420,7 +430,16 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co
if (storage_shared_key_credential)
result = std::make_unique<BlobContainerClient>(final_url, storage_shared_key_credential);
else
result = std::make_unique<BlobContainerClient>(final_url, managed_identity_credential);
{
/// If conneciton_url does not have '?', then its not SAS
if (pos == std::string::npos)
{
auto workload_identity_credential = std::make_shared<Azure::Identity::WorkloadIdentityCredential>();
result = std::make_unique<BlobContainerClient>(final_url, workload_identity_credential);
}
else
result = std::make_unique<BlobContainerClient>(final_url, managed_identity_credential);
}
}
else
{
@ -441,7 +460,16 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co
if (storage_shared_key_credential)
result = std::make_unique<BlobContainerClient>(final_url, storage_shared_key_credential);
else
result = std::make_unique<BlobContainerClient>(final_url, managed_identity_credential);
{
/// If conneciton_url does not have '?', then its not SAS
if (pos == std::string::npos)
{
auto workload_identity_credential = std::make_shared<Azure::Identity::WorkloadIdentityCredential>();
result = std::make_unique<BlobContainerClient>(final_url, workload_identity_credential);
}
else
result = std::make_unique<BlobContainerClient>(final_url, managed_identity_credential);
}
}
else
{

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int SUPPORT_IS_DISABLED;
extern const int ACCESS_DENIED;
extern const int DEPRECATED_FUNCTION;
};
enum class FunctionOrigin : int8_t
@ -152,7 +153,8 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c
|| e.code() == ErrorCodes::FUNCTION_NOT_ALLOWED
|| e.code() == ErrorCodes::NOT_IMPLEMENTED
|| e.code() == ErrorCodes::SUPPORT_IS_DISABLED
|| e.code() == ErrorCodes::ACCESS_DENIED)
|| e.code() == ErrorCodes::ACCESS_DENIED
|| e.code() == ErrorCodes::DEPRECATED_FUNCTION)
{
/// Ignore exception, show is_deterministic = NULL.
}

View File

@ -245,13 +245,20 @@ bool SystemRemoteDataPathsSource::nextFile()
try
{
const auto & disk = disks[current_disk].second;
const auto current_path = getCurrentPath();
/// Files or directories can disappear due to concurrent operations
if (!disk->exists(current_path))
continue;
/// Stop if current path is a file
if (disk->isFile(getCurrentPath()))
if (disk->isFile(current_path))
return true;
/// If current path is a directory list its contents and step into it
std::vector<std::string> children;
disk->listFiles(getCurrentPath(), children);
disk->listFiles(current_path, children);
/// Use current predicate for all children
const auto & skip_predicate = getCurrentSkipPredicate();
@ -271,6 +278,14 @@ bool SystemRemoteDataPathsSource::nextFile()
throw;
}
catch (const fs::filesystem_error & e)
{
/// Files or directories can disappear due to concurrent operations
if (e.code() == std::errc::no_such_file_or_directory)
continue;
throw;
}
}
}

View File

@ -348,7 +348,7 @@ class Backport:
repo: str,
fetch_from: Optional[str],
dry_run: bool,
must_create_backport_label: str,
must_create_backport_labels: List[str],
backport_created_label: str,
):
self.gh = gh
@ -356,7 +356,7 @@ class Backport:
self._fetch_from = fetch_from
self.dry_run = dry_run
self.must_create_backport_label = must_create_backport_label
self.must_create_backport_labels = must_create_backport_labels
self.backport_created_label = backport_created_label
self._remote = ""
@ -402,11 +402,18 @@ class Backport:
logging.info("Getting release PRs")
self.release_prs = self.gh.get_release_pulls(self._repo_name)
self.release_branches = [pr.head.ref for pr in self.release_prs]
self.labels_to_backport = [
f"v{branch}-must-backport" for branch in self.release_branches
]
if self._fetch_from:
if not self._fetch_from:
self.labels_to_backport = [
f"v{branch}-must-backport" for branch in self.release_branches
]
else:
fetch_release_prs = self.gh.get_release_pulls(self._fetch_from)
fetch_release_branches = [pr.head.ref for pr in fetch_release_prs]
self.labels_to_backport = [
f"v{branch}-must-backport" for branch in fetch_release_branches
]
logging.info("Fetching from %s", self._fetch_from)
fetch_from_repo = self.gh.get_repo(self._fetch_from)
git_runner(
@ -445,12 +452,12 @@ class Backport:
) - timedelta(days=reserve_search_days)
# To not have a possible TZ issues
tomorrow = date.today() + timedelta(days=1)
logging.info("Receive PRs suppose to be backported")
logging.info("Receive PRs supposed to be backported")
query_args = {
"query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}",
"label": ",".join(
self.labels_to_backport + [self.must_create_backport_label]
self.labels_to_backport + self.must_create_backport_labels
),
"merged": [since_date, tomorrow],
}
@ -473,12 +480,17 @@ class Backport:
def process_pr(self, pr: PullRequest) -> None:
pr_labels = [label.name for label in pr.labels]
if self.must_create_backport_label in pr_labels:
branches = [
ReleaseBranch(br, pr, self.repo, self.backport_created_label)
for br in self.release_branches
] # type: List[ReleaseBranch]
else:
for label in self.must_create_backport_labels:
# We backport any vXXX-must-backport to all branches of the fetch repo (better than no backport)
if label in pr_labels or self._fetch_from:
branches = [
ReleaseBranch(br, pr, self.repo, self.backport_created_label)
for br in self.release_branches
] # type: List[ReleaseBranch]
break
if not branches:
branches = [
ReleaseBranch(br, pr, self.repo, self.backport_created_label)
for br in [
@ -487,16 +499,16 @@ class Backport:
if label in self.labels_to_backport
]
]
if not branches:
# This is definitely some error. There must be at least one branch
# It also make the whole program exit code non-zero
self.error = Exception(
f"There are no branches to backport PR #{pr.number}, logical error"
)
raise self.error
if not branches:
# This is definitely some error. There must be at least one branch
# It also make the whole program exit code non-zero
self.error = Exception(
f"There are no branches to backport PR #{pr.number}, logical error"
)
raise self.error
logging.info(
" PR #%s is suppose to be backported to %s",
" PR #%s is supposed to be backported to %s",
pr.number,
", ".join(map(str, branches)),
)
@ -576,6 +588,7 @@ def parse_args():
default=Labels.MUST_BACKPORT,
choices=(Labels.MUST_BACKPORT, Labels.MUST_BACKPORT_CLOUD),
help="label to filter PRs to backport",
nargs="+",
)
parser.add_argument(
"--backport-created-label",
@ -647,7 +660,9 @@ def main():
args.repo,
args.from_repo,
args.dry_run,
args.must_create_backport_label,
args.must_create_backport_label
if isinstance(args.must_create_backport_label, list)
else [args.must_create_backport_label],
args.backport_created_label,
)
# https://github.com/python/mypy/issues/3004
@ -657,7 +672,7 @@ def main():
bp.receive_prs_for_backport(args.reserve_search_days)
bp.process_backports()
if bp.error is not None:
logging.error("Finished successfully, but errors occured!")
logging.error("Finished successfully, but errors occurred!")
raise bp.error

View File

@ -17,7 +17,7 @@ from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union
import docker_images_helper
import upload_result_helper
from build_check import get_release_or_pr
from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames
from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames, StatusNames
from ci_utils import GHActions, is_hex, normalize_string
from clickhouse_helper import (
CiLogsCredentials,
@ -32,15 +32,19 @@ from commit_status_helper import (
RerunHelper,
format_description,
get_commit,
get_commit_filtered_statuses,
post_commit_status,
set_status_comment,
trigger_mergeable_check,
update_mergeable_check,
)
from digest_helper import DockerDigester, JobDigester
from env_helper import (
CI,
GITHUB_JOB_API_URL,
GITHUB_REPOSITORY,
GITHUB_RUN_URL,
GITHUB_UPSTREAM_REPOSITORY,
REPO_COPY,
REPORT_PATH,
S3_BUILDS_BUCKET,
@ -51,8 +55,9 @@ from git_helper import GIT_PREFIX, Git
from git_helper import Runner as GitRunner
from github_helper import GitHub
from pr_info import PRInfo
from report import ERROR, SUCCESS, BuildResult, JobReport
from report import ERROR, SUCCESS, BuildResult, JobReport, get_status
from s3_helper import S3Helper
from synchronizer_utils import SYNC_BRANCH_PREFIX
from version_helper import get_version_from_repo
# pylint: disable=too-many-lines
@ -2106,6 +2111,7 @@ def main() -> int:
check_url = log_url
else:
# test job
gh = GitHub(get_best_robot_token(), per_page=100)
additional_urls = []
s3_path_prefix = "/".join(
(
@ -2133,9 +2139,7 @@ def main() -> int:
job_report.check_name or _get_ext_check_name(args.job_name),
additional_urls=additional_urls or None,
)
commit = get_commit(
GitHub(get_best_robot_token(), per_page=100), pr_info.sha
)
commit = get_commit(gh, pr_info.sha)
post_commit_status(
commit,
job_report.status,
@ -2147,12 +2151,49 @@ def main() -> int:
)
if not pr_info.is_merge_queue:
# in the merge queue mergeable status must be set only in FinishCheck (last job in wf)
update_mergeable_check(
mergeable_status = update_mergeable_check(
commit,
pr_info,
job_report.check_name or _get_ext_check_name(args.job_name),
)
# Process upstream StatusNames.SYNC
if (
pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/")
and mergeable_status
and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY
):
pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1])
upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY)
head_sha = upstream_repo.get_pull(pr_number).head.sha
upstream_commit = upstream_repo.get_commit(head_sha)
post_commit_status(
upstream_commit,
get_status(mergeable_status.state),
"", # let's won't expose any urls from cloud
mergeable_status.description,
StatusNames.SYNC,
)
trigger_mergeable_check(
upstream_commit,
get_commit_filtered_statuses(upstream_commit),
True,
)
prepared_events = prepare_tests_results_for_clickhouse(
pr_info,
[],
job_report.status,
0,
job_report.start_time,
f"https://github.com/ClickHouse/ClickHouse/pull/{pr_number}",
StatusNames.SYNC,
)
prepared_events[0]["test_context_raw"] = args.job_name
ch_helper.insert_events_into(
db="default", table="checks", events=prepared_events
)
print(f"Job report url: [{check_url}]")
prepared_events = prepare_tests_results_for_clickhouse(
pr_info,

View File

@ -85,13 +85,13 @@ def post_commit_status(
check_name: Optional[str] = None,
pr_info: Optional[PRInfo] = None,
dump_to_file: bool = False,
) -> None:
) -> CommitStatus:
"""The parameters are given in the same order as for commit.create_status,
if an optional parameter `pr_info` is given, the `set_status_comment` functions
is invoked to add or update the comment with statuses overview"""
for i in range(RETRY):
try:
commit.create_status(
commit_status = commit.create_status(
state=state,
target_url=report_url if report_url is not None else NotSet,
description=description if description is not None else NotSet,
@ -128,6 +128,8 @@ def post_commit_status(
pr_num=pr_info.number,
).dump_status()
return commit_status
STATUS_ICON_MAP = defaultdict(
str,
@ -425,16 +427,23 @@ def set_mergeable_check(
commit: Commit,
description: str = "",
state: StatusType = SUCCESS,
) -> None:
commit.create_status(
context=StatusNames.MERGEABLE,
description=format_description(description),
state=state,
target_url=GITHUB_RUN_URL,
hide_url: bool = False,
) -> CommitStatus:
report_url = GITHUB_RUN_URL
if hide_url:
report_url = ""
return post_commit_status(
commit,
state,
report_url,
format_description(description),
StatusNames.MERGEABLE,
)
def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> None:
def update_mergeable_check(
commit: Commit, pr_info: PRInfo, check_name: str
) -> Optional[CommitStatus]:
"check if the check_name in REQUIRED_CHECKS and then trigger update"
not_run = (
pr_info.labels.intersection({Labels.SKIP_MERGEABLE_CHECK, Labels.RELEASE})
@ -445,15 +454,17 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) ->
if not_run:
# Let's avoid unnecessary work
return
return None
logging.info("Update Mergeable Check by %s", check_name)
statuses = get_commit_filtered_statuses(commit)
trigger_mergeable_check(commit, statuses)
return trigger_mergeable_check(commit, statuses)
def trigger_mergeable_check(commit: Commit, statuses: CommitStatuses) -> None:
def trigger_mergeable_check(
commit: Commit, statuses: CommitStatuses, hide_url: bool = False
) -> CommitStatus:
"""calculate and update StatusNames.MERGEABLE"""
required_checks = [
status for status in statuses if status.context in REQUIRED_CHECKS
@ -486,4 +497,6 @@ def trigger_mergeable_check(commit: Commit, statuses: CommitStatuses) -> None:
description = format_description(description)
if mergeable_status is None or mergeable_status.description != description:
set_mergeable_check(commit, description, state)
return set_mergeable_check(commit, description, state, hide_url)
return mergeable_status

View File

@ -22,6 +22,9 @@ GITHUB_JOB = os.getenv("GITHUB_JOB_OVERRIDDEN", "") or os.getenv("GITHUB_JOB", "
GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")
GITHUB_RUN_ID = os.getenv("GITHUB_RUN_ID", "0")
GITHUB_SERVER_URL = os.getenv("GITHUB_SERVER_URL", "https://github.com")
GITHUB_UPSTREAM_REPOSITORY = os.getenv(
"GITHUB_UPSTREAM_REPOSITORY", "ClickHouse/ClickHouse"
)
GITHUB_WORKSPACE = os.getenv("GITHUB_WORKSPACE", git_root)
GITHUB_RUN_URL = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}"
IMAGES_PATH = os.getenv("IMAGES_PATH", TEMP_PATH)

View File

@ -158,7 +158,7 @@ class PRInfo:
else:
self.sha = github_event["pull_request"]["head"]["sha"]
self.commit_html_url = f"{repo_prefix}/commits/{self.sha}"
self.commit_html_url = f"{repo_prefix}/commit/{self.sha}"
self.pr_html_url = f"{repo_prefix}/pull/{self.number}"
# master or backport/xx.x/xxxxx - where the PR will be merged
@ -213,7 +213,7 @@ class PRInfo:
.replace("{base}", base_sha)
.replace("{head}", self.sha)
)
self.commit_html_url = f"{repo_prefix}/commits/{self.sha}"
self.commit_html_url = f"{repo_prefix}/commit/{self.sha}"
elif "commits" in github_event:
self.event_type = EventType.PUSH
@ -227,7 +227,7 @@ class PRInfo:
logging.error("Failed to convert %s to integer", merged_pr)
self.sha = github_event["after"]
pull_request = get_pr_for_commit(self.sha, github_event["ref"])
self.commit_html_url = f"{repo_prefix}/commits/{self.sha}"
self.commit_html_url = f"{repo_prefix}/commit/{self.sha}"
if pull_request is None or pull_request["state"] == "closed":
# it's merged PR to master
@ -296,7 +296,7 @@ class PRInfo:
"GITHUB_SHA", "0000000000000000000000000000000000000000"
)
self.number = 0
self.commit_html_url = f"{repo_prefix}/commits/{self.sha}"
self.commit_html_url = f"{repo_prefix}/commit/{self.sha}"
self.pr_html_url = f"{repo_prefix}/commits/{ref}"
self.base_ref = ref
self.base_name = self.repo_full_name

View File

@ -49,6 +49,15 @@ def _state_rank(status: str) -> int:
return 3
def get_status(status: str) -> StatusType:
"function to get the StatusType for a status or ERROR"
try:
ind = STATUSES.index(status) # type: ignore
return STATUSES[ind]
except ValueError:
return ERROR
def get_worst_status(statuses: Iterable[str]) -> StatusType:
worst_status = SUCCESS # type: StatusType
for status in statuses:

View File

@ -0,0 +1,4 @@
#!/usr/bin/env python
SYNC_BRANCH_PREFIX = "sync-upstream"
SYNC_MASTER_BRANCH = f"{SYNC_BRANCH_PREFIX}/master"

View File

@ -1,6 +1,7 @@
<clickhouse>
<merge_tree>
<!-- Setting randomized for stress test, it is disabled here and this line is used to randomize it in the script -->
<allow_experimental_block_number_column>0</allow_experimental_block_number_column>
<enable_block_number_column>0</enable_block_number_column>
<enable_block_offset_column>0</enable_block_offset_column>
</merge_tree>
</clickhouse>

View File

@ -165,29 +165,6 @@ def put_azure_file_content(filename, port, data):
blob_client.upload_blob(buf)
@pytest.fixture(autouse=True, scope="function")
def delete_all_files(cluster):
port = cluster.env_variables["AZURITE_PORT"]
connection_string = (
f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;"
f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;"
f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;"
)
blob_service_client = BlobServiceClient.from_connection_string(connection_string)
containers = blob_service_client.list_containers()
for container in containers:
container_client = blob_service_client.get_container_client(container)
blob_list = container_client.list_blobs()
for blob in blob_list:
print(blob)
blob_client = container_client.get_blob_client(blob)
blob_client.delete_blob()
assert len(list(container_client.list_blobs())) == 0
yield
def test_backup_restore(cluster):
node = cluster.instances["node"]
port = cluster.env_variables["AZURITE_PORT"]

View File

@ -20,6 +20,8 @@ SELECT
length(message_format_string) = 0
AND message not like '% Received from %clickhouse-staging.com:9440%'
AND source_file not like '%/AWSLogger.cpp%'
AND source_file not like '%/BaseDaemon.cpp%'
AND logger_name not in ('RaftInstance')
GROUP BY message ORDER BY c LIMIT 10
))
FROM logs
@ -129,7 +131,10 @@ create temporary table known_short_messages (s String) as select * from (select
'String size is too big ({}), maximum: {}',
'Substitution {} is not set',
'Table {} does not exist',
'Table {} doesn\'t exist',
'Table {}.{} doesn\'t exist',
'Table {} doesn\'t exist',
'Table {} is not empty',
'There are duplicate id {}',
'There is no cache by name: {}',
'Too large node state size',
@ -139,9 +144,11 @@ create temporary table known_short_messages (s String) as select * from (select
'Unknown BSON type: {}',
'Unknown explain kind \'{}\'',
'Unknown format {}',
'Unknown geometry type {}',
'Unknown identifier: \'{}\'',
'Unknown input format {}',
'Unknown setting {}',
'Unknown setting \'{}\'',
'Unknown statistic column: {}',
'Unknown table function {}',
'User has been dropped',
@ -216,7 +223,10 @@ select 'noisy Debug messages',
-- Same as above for Info
WITH 0.05 as threshold
select 'noisy Info messages',
greatest(coalesce(((select message_format_string, count() from logs where level = 'Information' group by message_format_string order by count() desc limit 1) as top_message).2, 0) / (select count() from logs), threshold) as r,
greatest(coalesce(((select message_format_string, count() from logs
where level = 'Information'
and message_format_string not in ('Sorting and writing part of data into temporary file {}', 'Done writing part of data into temporary file {}, compressed {}, uncompressed {}')
group by message_format_string order by count() desc limit 1) as top_message).2, 0) / (select count() from logs), threshold) as r,
r <= threshold ? '' : top_message.1;
-- Same as above for Warning

View File

@ -1,4 +1,5 @@
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
SET max_bytes_before_external_group_by = 0;
SET allow_deprecated_functions = 1;
SELECT k, finalizeAggregation(sum_state), runningAccumulate(sum_state) FROM (SELECT intDiv(number, 50000) AS k, sumState(number) AS sum_state FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k);

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS arena;
CREATE TABLE arena (k UInt8, d String) ENGINE = Memory;
INSERT INTO arena SELECT number % 10 AS k, hex(intDiv(number, 10) % 1000) AS d FROM system.numbers LIMIT 10000000;

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
select runningDifference(x) from (select arrayJoin([0, 1, 5, 10]) as x);
select '-';
select runningDifference(x) from (select arrayJoin([2, Null, 3, Null, 10]) as x);

View File

@ -0,0 +1,16 @@
drop table if exists rate_test;
drop table if exists rate_test2;
create table rate_test (timestamp UInt32, event UInt32) engine=Memory;
insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008);
create table rate_test2 (timestamp UInt32, event UInt32) engine=Memory;
SELECT boundingRatioMerge(state) FROM (
select boundingRatioState(timestamp, event) as state from rate_test
UNION ALL
SELECT boundingRatioState(timestamp, event) FROM rate_test2 WHERE 1=0
);
drop table if exists rate_test;
drop table if exists rate_test2;

View File

@ -1,5 +1,6 @@
SET send_logs_level = 'fatal';
SET convert_query_to_cnf = 0;
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS test_00808;
CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date);

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
-- no arguments
select neighbor(); -- { serverError 42 }
-- single argument
@ -39,4 +40,4 @@ select number, -number * 2 as offset, neighbor(number, offset, number * 10) from
select 'Dynamic column and offset, without defaults';
select number, -(number - 2) * 2 as offset, neighbor(number, offset) from numbers(6);
select 'Constant column';
select number, neighbor(1000, 10) from numbers(3);
select number, neighbor(1000, 10) from numbers(3);

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
SELECT number, neighbor(toString(number), 0) FROM numbers(10);
SELECT number, neighbor(toString(number), 5) FROM numbers(10);

View File

@ -1,5 +1,6 @@
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
SET max_bytes_before_external_group_by = 0;
SET allow_deprecated_functions = 1;
SELECT grouping,
item,
@ -11,4 +12,4 @@ FROM (
FROM (SELECT number FROM system.numbers LIMIT 30)
GROUP BY grouping, item
ORDER BY grouping, item
);
);

View File

@ -1,4 +1,6 @@
SET allow_deprecated_functions = 1;
SELECT runningAccumulate(string_state)
FROM (
SELECT argMaxState(repeat('a', 48), 1) AS string_state
)
)

View File

@ -1,6 +1,7 @@
SET enable_optimize_predicate_expression = 1;
SET joined_subquery_requires_alias = 0;
SET convert_query_to_cnf = 0;
SET allow_deprecated_functions = 1;
-- https://github.com/ClickHouse/ClickHouse/issues/3885
-- https://github.com/ClickHouse/ClickHouse/issues/5485

View File

@ -1,2 +1,3 @@
SET allow_deprecated_functions = 1;
SELECT neighbor(toString(number), -9223372036854775808) FROM numbers(100); -- { serverError 69 }
WITH neighbor(toString(number), toInt64(rand64())) AS x SELECT * FROM system.numbers WHERE NOT ignore(x); -- { serverError 69 }

View File

@ -1,4 +1,5 @@
SET max_insert_threads = 1, max_threads = 100, min_insert_block_size_rows = 1048576, max_block_size = 65536;
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS t;
CREATE TABLE t (x UInt64) ENGINE = StripeLog;
-- For trivial INSERT SELECT, max_threads is lowered to max_insert_threads and max_block_size is changed to min_insert_block_size_rows.

View File

@ -1 +1,2 @@
SET allow_deprecated_functions = 1;
SELECT k, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM (SELECT arrayJoin([(NULL, 65535), ('a', 7), ('a', 3), ('b', 11), ('b', 2), ('', -9223372036854775808)]) AS t)) WHERE i = 9223372036854775807;

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
SET output_format_pretty_row_numbers = 0;
SELECT

View File

@ -478,6 +478,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1
-- explain
Expression (Projection)
Sorting (Sorting for ORDER BY)

View File

@ -314,7 +314,8 @@ FROM
FROM numbers(10)
ORDER BY number DESC
)
ORDER BY number ASC"
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1"
run_query "$query"
echo "-- non-stateful function does _not_ prevent removing inner ORDER BY"

View File

@ -477,6 +477,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1
-- explain
Expression (Project names)
Sorting (Sorting for ORDER BY)

View File

@ -1,3 +1,5 @@
SET allow_deprecated_functions = 1;
DROP TABLE IF EXISTS session_events;
DROP TABLE IF EXISTS event_types;

View File

@ -1,3 +1,4 @@
SET allow_deprecated_functions = 1;
drop table if exists largestTriangleThreeBucketsTestFloat64Float64;
CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64
@ -60,4 +61,4 @@ SELECT
point_x - neighbor(point_x, -1) AS point_x_diff_with_previous_row
FROM largestTriangleTreeBucketsBucketSizeTest LIMIT 990, 10;
DROP TABLE largestTriangleTreeBucketsBucketSizeTest;
DROP TABLE largestTriangleTreeBucketsBucketSizeTest;

View File

@ -1,3 +1,5 @@
SET allow_deprecated_functions = 1;
CREATE TABLE t
(
`rDate` String,

View File

@ -1,4 +1,4 @@
-- Tags: no-replicated-database, no-fasttest
-- Tags: no-replicated-database, no-fasttest, no-parallel, no-random-settings, no-random-merge-tree-settings
DROP TABLE IF EXISTS 03000_traverse_shadow_system_data_path_table;

View File

@ -0,0 +1,53 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-shared-merge-tree
# Tag no-fasttest: requires S3
# Tag no-shared-merge-tree: does not support replication
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt"
${CLICKHOUSE_CLIENT} -nm --query "
create table test_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = disk(
name = s3_plain_rewritable,
type = s3_plain_rewritable,
endpoint = 'http://localhost:11111/test/test_mt/',
access_key_id = clickhouse,
secret_access_key = clickhouse);
"
${CLICKHOUSE_CLIENT} -nm --query "
insert into test_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
insert into test_mt (*) select number, number, number from numbers_mt(10000);
select count(*) from test_mt;
select (*) from test_mt order by tuple(a, b) limit 10;
"
${CLICKHOUSE_CLIENT} --query "optimize table test_mt final"
${CLICKHOUSE_CLIENT} -m --query "
alter table test_mt add projection test_mt_projection (select * order by b)" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
${CLICKHOUSE_CLIENT} -nm --query "
alter table test_mt update c = 0 where a % 2 = 1;
alter table test_mt add column d Int64 after c;
alter table test_mt drop column c;
" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
${CLICKHOUSE_CLIENT} -nm --query "
detach table test_mt;
attach table test_mt;
"
${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt_dst"
${CLICKHOUSE_CLIENT} -m --query "
create table test_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = 's3_plain_rewritable'
"
${CLICKHOUSE_CLIENT} -m --query "
alter table test_mt move partition 0 to table test_mt_dst" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"

View File

@ -1,35 +0,0 @@
-- Tags: no-fasttest
-- Tag: no-fasttest -- requires S3
drop table if exists test_mt;
create table test_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = disk(
name = s3_plain_rewritable,
type = s3_plain_rewritable,
endpoint = 'http://localhost:11111/test/test_mt/',
access_key_id = clickhouse,
secret_access_key = clickhouse);
insert into test_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
insert into test_mt (*) select number, number, number from numbers_mt(10000);
select count(*) from test_mt;
select (*) from test_mt order by tuple(a, b) limit 10;
optimize table test_mt final;
alter table test_mt add projection test_mt_projection (
select * order by b); -- { serverError SUPPORT_IS_DISABLED }
alter table test_mt update c = 0 where a % 2 = 1; -- { serverError SUPPORT_IS_DISABLED }
alter table test_mt add column d Int64 after c; -- { serverError SUPPORT_IS_DISABLED }
alter table test_mt drop column c; -- { serverError SUPPORT_IS_DISABLED }
detach table test_mt;
attach table test_mt;
drop table if exists test_mt_dst;
create table test_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = 's3_plain_rewritable';
alter table test_mt move partition 0 to table test_mt_dst; -- { serverError SUPPORT_IS_DISABLED }

View File

@ -5,7 +5,7 @@ Arrow
a UInt64
a_nullable Nullable(UInt64)
Parquet
b Array(Nullable(UInt64))
b Array(UInt64)
b_nullable Array(Nullable(UInt64))
Arrow
b Array(Nullable(UInt64))
@ -21,13 +21,13 @@ d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n
Arrow
d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String))))
Parquet
e Map(UInt64, Nullable(String))
e Map(UInt64, String)
e_nullable Map(UInt64, Nullable(String))
Arrow
e Map(UInt64, Nullable(String))
e_nullable Map(UInt64, Nullable(String))
Parquet
f Map(UInt64, Map(UInt64, Nullable(String)))
f Map(UInt64, Map(UInt64, String))
f_nullables Map(UInt64, Map(UInt64, Nullable(String)))
Arrow
f Map(UInt64, Map(UInt64, Nullable(String)))

View File

@ -1,5 +0,0 @@
SELECT serverUUID() AS s, count() FROM remote('127.0.0.{1,2}', system.one) GROUP BY s format Null;
select getMacro('replica') as s, count() from remote('127.0.0.{1,2}', system.one) group by s;
select uptime() as s, count() FROM remote('127.0.0.{1,2}', system.one) group by s format Null;

View File

@ -0,0 +1,13 @@
SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -- { serverError 721 }
SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError 721 }
SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); -- { serverError 721 }
SET allow_deprecated_functions=1;
SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10 FORMAT Null;
SELECT runningDifference(number) FROM system.numbers LIMIT 10 FORMAT Null;
SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k) FORMAT Null;

View File

@ -0,0 +1,2 @@
SET allow_experimental_analyzer=1;
SELECT untuple(x -> 0) -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

View File

@ -0,0 +1 @@
0 100

View File

@ -0,0 +1,11 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
query_id_prefix=${CLICKHOUSE_DATABASE}_test_benchmark
$CLICKHOUSE_BENCHMARK -i 100 -c 8 <<< "SELECT 1" --query_id_prefix $query_id_prefix 2>/dev/null
$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT --query "SELECT countIf(query_id = '$query_id_prefix'), countIf(query_id LIKE '$query_id_prefix%') FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish'"

View File

@ -0,0 +1,2 @@
1 2 3 0.3 1 2 4 0.3
5 6 7 0.4 5 6 8 0.4

View File

@ -0,0 +1,43 @@
-- https://github.com/ClickHouse/ClickHouse/issues/56287
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS tmp_a;
DROP TABLE IF EXISTS tmp_b;
CREATE TEMPORARY TABLE IF NOT EXISTS tmp_a
(
k1 Int32,
k2 Int32,
d1 Int32,
d2 Int32
) ENGINE = Memory;
INSERT INTO tmp_a VALUES (1,2,3,4);
INSERT INTO tmp_a VALUES (5,6,7,8);
CREATE TEMPORARY TABLE IF NOT EXISTS tmp_b (
k1 Int32,
k2 Int32,
d0 Float64
) ENGINE = Memory;
INSERT INTO tmp_b VALUES (1,2,0.3);
INSERT INTO tmp_b VALUES (5,6,0.4);
SELECT tb1.*,tb2.*
FROM
(
with tmp0 as (select k1,k2,d1 from tmp_a),
tmp_s as (select k1,k2,d0 from tmp_b),
tmp1 as (select tmp0.*,tmp_s.d0 from tmp0 left join tmp_s on tmp0.k1=tmp_s.k1 and tmp0.k2=tmp_s.k2)
select * from tmp1
) as tb1
LEFT JOIN
(
with tmp0 as (select k1,k2,d2 from tmp_a),
tmp_s as (select k1,k2,d0 from tmp_b),
tmp1 as (select tmp0.*,tmp_s.d0 from tmp0 left join tmp_s on tmp0.k1=tmp_s.k1 and tmp0.k2=tmp_s.k2)
select * from tmp1
) as tb2
ON tb1.k1=tb2.k1 AND tb1.k2=tb2.k2
ORDER BY k1;
DROP TABLE IF EXISTS tmp_a;
DROP TABLE IF EXISTS tmp_b;

View File

@ -0,0 +1,8 @@
-- https://github.com/ClickHouse/ClickHouse/issues/63264
SELECT count()
FROM remote(test_cluster_two_shards, system, one)
GROUP BY 'hi'
SETTINGS
allow_experimental_analyzer = 1,
group_by_two_level_threshold = 1,
group_by_two_level_threshold_bytes = 33950592;

View File

@ -1 +1,3 @@
SET allow_deprecated_functions = 1;
SELECT EventDate, finalizeAggregation(state), runningAccumulate(state) FROM (SELECT EventDate, uniqState(UserID) AS state FROM test.hits GROUP BY EventDate ORDER BY EventDate);

View File

@ -36,6 +36,7 @@ v23.9.4.11-stable 2023-11-08
v23.9.3.12-stable 2023-10-31
v23.9.2.56-stable 2023-10-19
v23.9.1.1854-stable 2023-09-29
v23.8.14.6-lts 2024-05-02
v23.8.13.25-lts 2024-04-26
v23.8.12.13-lts 2024-03-26
v23.8.11.28-lts 2024-03-15

1 v24.4.1.2088-stable 2024-05-01
36 v23.9.3.12-stable 2023-10-31
37 v23.9.2.56-stable 2023-10-19
38 v23.9.1.1854-stable 2023-09-29
39 v23.8.14.6-lts 2024-05-02
40 v23.8.13.25-lts 2024-04-26
41 v23.8.12.13-lts 2024-03-26
42 v23.8.11.28-lts 2024-03-15