mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into revert-65361-revert-65164-ecs
This commit is contained in:
commit
fd8c99241b
@ -42,19 +42,9 @@ endif ()
|
||||
# But use 2 parallel jobs, since:
|
||||
# - this is what llvm does
|
||||
# - and I've verfied that lld-11 does not use all available CPU time (in peak) while linking one binary
|
||||
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO)
|
||||
if (ARCH_AARCH64)
|
||||
# aarch64 builds start to often fail with OOMs (reason not yet clear), for now let's limit the concurrency
|
||||
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 1.")
|
||||
set (PARALLEL_LINK_JOBS 1)
|
||||
if (LINKER_NAME MATCHES "lld")
|
||||
math(EXPR LTO_JOBS ${NUMBER_OF_LOGICAL_CORES}/4)
|
||||
set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -Wl,--thinlto-jobs=${LTO_JOBS}")
|
||||
endif()
|
||||
elseif (PARALLEL_LINK_JOBS GREATER 2)
|
||||
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.")
|
||||
set (PARALLEL_LINK_JOBS 2)
|
||||
endif ()
|
||||
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLEL_LINK_JOBS GREATER 2)
|
||||
message(STATUS "ThinLTO provides its own parallel linking - limiting parallel link jobs to 2.")
|
||||
set (PARALLEL_LINK_JOBS 2)
|
||||
endif()
|
||||
|
||||
message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB RAM, 'OFF' means the native core count).")
|
||||
|
@ -66,6 +66,8 @@
|
||||
/// A minimal file used when the keeper is run without installation
|
||||
INCBIN(keeper_resource_embedded_xml, SOURCE_DIR "/programs/keeper/keeper_embedded.xml");
|
||||
|
||||
extern const char * GIT_HASH;
|
||||
|
||||
int mainEntryClickHouseKeeper(int argc, char ** argv)
|
||||
{
|
||||
DB::Keeper app;
|
||||
@ -675,7 +677,7 @@ void Keeper::logRevision() const
|
||||
"Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}",
|
||||
VERSION_STRING,
|
||||
ClickHouseRevision::getVersionRevision(),
|
||||
git_hash.empty() ? "<unknown>" : git_hash,
|
||||
GIT_HASH,
|
||||
build_id.empty() ? "<unknown>" : build_id,
|
||||
getpid());
|
||||
}
|
||||
|
@ -367,7 +367,7 @@ std::string LocalServer::getInitialCreateTableQuery()
|
||||
else
|
||||
table_structure = "(" + table_structure + ")";
|
||||
|
||||
return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});",
|
||||
return fmt::format("CREATE TEMPORARY TABLE {} {} ENGINE = File({}, {});",
|
||||
table_name, table_structure, data_format, table_file);
|
||||
}
|
||||
|
||||
|
@ -82,7 +82,8 @@ public:
|
||||
|
||||
Result authenticate(const String & user_name, const String & password) const
|
||||
{
|
||||
Poco::Net::HTTPRequest request{Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery()};
|
||||
Poco::Net::HTTPRequest request{
|
||||
Poco::Net::HTTPRequest::HTTP_GET, this->getURI().getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1};
|
||||
Poco::Net::HTTPBasicCredentials basic_credentials{user_name, password};
|
||||
basic_credentials.authenticate(request);
|
||||
|
||||
|
@ -18,13 +18,17 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_SET_SIGNAL_HANDLER;
|
||||
extern const int CANNOT_SEND_SIGNAL;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
extern const char * GIT_HASH;
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
@ -334,7 +338,7 @@ void SignalListener::onTerminate(std::string_view message, UInt32 thread_num) co
|
||||
size_t pos = message.find('\n');
|
||||
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", thread_num, message.substr(0, pos));
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, thread_num, message.substr(0, pos));
|
||||
|
||||
/// Print trace from std::terminate exception line-by-line to make it easy for grep.
|
||||
while (pos != std::string_view::npos)
|
||||
@ -368,7 +372,7 @@ try
|
||||
|
||||
LOG_FATAL(log, "########## Short fault info ############");
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
|
||||
thread_num, sig);
|
||||
|
||||
std::string signal_description = "Unknown signal";
|
||||
@ -434,13 +438,13 @@ try
|
||||
if (query_id.empty())
|
||||
{
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
|
||||
thread_num, signal_description, sig);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
|
||||
thread_num, query_id, query, signal_description, sig);
|
||||
}
|
||||
|
||||
|
@ -452,8 +452,6 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
|
||||
build_id = SymbolIndex::instance().getBuildIDHex();
|
||||
#endif
|
||||
|
||||
git_hash = GIT_HASH;
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
std::string executable_path = getExecutablePath();
|
||||
|
||||
@ -466,7 +464,7 @@ void BaseDaemon::logRevision() const
|
||||
{
|
||||
logger().information("Starting " + std::string{VERSION_FULL}
|
||||
+ " (revision: " + std::to_string(ClickHouseRevision::getVersionRevision())
|
||||
+ ", git hash: " + (git_hash.empty() ? "<unknown>" : git_hash)
|
||||
+ ", git hash: " + std::string(GIT_HASH)
|
||||
+ ", build id: " + (build_id.empty() ? "<unknown>" : build_id) + ")"
|
||||
+ ", PID " + std::to_string(getpid()));
|
||||
}
|
||||
|
@ -165,7 +165,6 @@ protected:
|
||||
Poco::Util::AbstractConfiguration * last_configuration = nullptr;
|
||||
|
||||
String build_id;
|
||||
String git_hash;
|
||||
String stored_binary_hash;
|
||||
|
||||
bool should_setup_watchdog = false;
|
||||
|
@ -115,7 +115,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix(
|
||||
dynamic_state->max_dynamic_types = column_dynamic.getMaxDynamicTypes();
|
||||
/// Write max_dynamic_types parameter, because it can differ from the max_dynamic_types
|
||||
/// that is specified in the Dynamic type (we could decrease it before merge).
|
||||
writeBinaryLittleEndian(dynamic_state->max_dynamic_types, *stream);
|
||||
writeVarUInt(dynamic_state->max_dynamic_types, *stream);
|
||||
|
||||
dynamic_state->variant_type = variant_info.variant_type;
|
||||
dynamic_state->variant_names = variant_info.variant_names;
|
||||
@ -123,7 +123,7 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix(
|
||||
|
||||
/// Write information about variants.
|
||||
size_t num_variants = dynamic_state->variant_names.size() - 1; /// Don't write shared variant, Dynamic column should always have it.
|
||||
writeBinaryLittleEndian(num_variants, *stream);
|
||||
writeVarUInt(num_variants, *stream);
|
||||
if (settings.data_types_binary_encoding)
|
||||
{
|
||||
const auto & variants = assert_cast<const DataTypeVariant &>(*dynamic_state->variant_type).getVariants();
|
||||
@ -252,11 +252,11 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD
|
||||
readBinaryLittleEndian(structure_version, *structure_stream);
|
||||
auto structure_state = std::make_shared<DeserializeBinaryBulkStateDynamicStructure>(structure_version);
|
||||
/// Read max_dynamic_types parameter.
|
||||
readBinaryLittleEndian(structure_state->max_dynamic_types, *structure_stream);
|
||||
readVarUInt(structure_state->max_dynamic_types, *structure_stream);
|
||||
/// Read information about variants.
|
||||
DataTypes variants;
|
||||
size_t num_variants;
|
||||
readBinaryLittleEndian(num_variants, *structure_stream);
|
||||
readVarUInt(num_variants, *structure_stream);
|
||||
variants.reserve(num_variants + 1); /// +1 for shared variant.
|
||||
if (settings.data_types_binary_encoding)
|
||||
{
|
||||
|
@ -199,7 +199,7 @@ void SerializationObject::serializeBinaryBulkStatePrefix(
|
||||
auto object_state = std::make_shared<SerializeBinaryBulkStateObject>(serialization_version);
|
||||
object_state->max_dynamic_paths = column_object.getMaxDynamicPaths();
|
||||
/// Write max_dynamic_paths parameter.
|
||||
writeBinaryLittleEndian(object_state->max_dynamic_paths, *stream);
|
||||
writeVarUInt(object_state->max_dynamic_paths, *stream);
|
||||
/// Write all dynamic paths in sorted order.
|
||||
object_state->sorted_dynamic_paths.reserve(dynamic_paths.size());
|
||||
for (const auto & [path, _] : dynamic_paths)
|
||||
@ -354,7 +354,7 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationObject::deserializeOb
|
||||
readBinaryLittleEndian(serialization_version, *structure_stream);
|
||||
auto structure_state = std::make_shared<DeserializeBinaryBulkStateObjectStructure>(serialization_version);
|
||||
/// Read max_dynamic_paths parameter.
|
||||
readBinaryLittleEndian(structure_state->max_dynamic_paths, *structure_stream);
|
||||
readVarUInt(structure_state->max_dynamic_paths, *structure_stream);
|
||||
/// Read the sorted list of dynamic paths.
|
||||
size_t dynamic_paths_size;
|
||||
readVarUInt(dynamic_paths_size, *structure_stream);
|
||||
|
@ -52,7 +52,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
|
||||
|
||||
void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/)
|
||||
{
|
||||
iterateMetadataFiles(local_context, [this, &local_context](const String & file_name)
|
||||
iterateMetadataFiles([this, &local_context](const String & file_name)
|
||||
{
|
||||
const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4));
|
||||
|
||||
|
@ -12,7 +12,7 @@ class DatabaseLazyIterator;
|
||||
class Context;
|
||||
|
||||
/** Lazy engine of databases.
|
||||
* Works like DatabaseOrdinary, but stores in memory only the cache.
|
||||
* Works like DatabaseOrdinary, but stores only recently accessed tables in memory.
|
||||
* Can be used only with *Log engines.
|
||||
*/
|
||||
class DatabaseLazy final : public DatabaseOnDisk
|
||||
|
@ -568,14 +568,14 @@ void DatabaseOnDisk::drop(ContextPtr local_context)
|
||||
assert(TSA_SUPPRESS_WARNING_FOR_READ(tables).empty());
|
||||
if (local_context->getSettingsRef().force_remove_data_recursively_on_drop)
|
||||
{
|
||||
(void)fs::remove_all(local_context->getPath() + getDataPath());
|
||||
(void)fs::remove_all(std::filesystem::path(getContext()->getPath()) / data_path);
|
||||
(void)fs::remove_all(getMetadataPath());
|
||||
}
|
||||
else
|
||||
{
|
||||
try
|
||||
{
|
||||
(void)fs::remove(local_context->getPath() + getDataPath());
|
||||
(void)fs::remove(std::filesystem::path(getContext()->getPath()) / data_path);
|
||||
(void)fs::remove(getMetadataPath());
|
||||
}
|
||||
catch (const fs::filesystem_error & e)
|
||||
@ -613,7 +613,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const IteratingFunction & process_metadata_file) const
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const
|
||||
{
|
||||
auto process_tmp_drop_metadata_file = [&](const String & file_name)
|
||||
{
|
||||
@ -621,7 +621,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
|
||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
||||
const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext));
|
||||
|
||||
if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name))
|
||||
if (fs::exists(std::filesystem::path(getContext()->getPath()) / data_path / object_name))
|
||||
{
|
||||
fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql");
|
||||
LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name));
|
||||
@ -638,7 +638,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
|
||||
std::vector<std::pair<String, bool>> metadata_files;
|
||||
|
||||
fs::directory_iterator dir_end;
|
||||
for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
|
||||
for (fs::directory_iterator dir_it(metadata_path); dir_it != dir_end; ++dir_it)
|
||||
{
|
||||
String file_name = dir_it->path().filename();
|
||||
/// For '.svn', '.gitignore' directory and similar.
|
||||
|
@ -64,7 +64,7 @@ public:
|
||||
time_t getObjectMetadataModificationTime(const String & object_name) const override;
|
||||
|
||||
String getDataPath() const override { return data_path; }
|
||||
String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; }
|
||||
String getTableDataPath(const String & table_name) const override { return std::filesystem::path(data_path) / escapeForFileName(table_name) / ""; }
|
||||
String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.getTable()); }
|
||||
String getMetadataPath() const override { return metadata_path; }
|
||||
|
||||
@ -83,7 +83,7 @@ protected:
|
||||
|
||||
using IteratingFunction = std::function<void(const String &)>;
|
||||
|
||||
void iterateMetadataFiles(ContextPtr context, const IteratingFunction & process_metadata_file) const;
|
||||
void iterateMetadataFiles(const IteratingFunction & process_metadata_file) const;
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(
|
||||
const String & table_name,
|
||||
|
@ -55,7 +55,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
static constexpr const char * const CONVERT_TO_REPLICATED_FLAG_NAME = "convert_to_replicated";
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_)
|
||||
: DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_)
|
||||
: DatabaseOrdinary(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseOrdinary (" + name_ + ")", context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -265,7 +265,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
|
||||
}
|
||||
};
|
||||
|
||||
iterateMetadataFiles(local_context, process_metadata);
|
||||
iterateMetadataFiles(process_metadata);
|
||||
|
||||
size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count;
|
||||
size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries;
|
||||
|
@ -853,9 +853,10 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
/// Parse numbers (including decimals), strings, arrays and tuples of them.
|
||||
|
||||
Pos begin = pos;
|
||||
const char * data_begin = pos->begin;
|
||||
const char * data_end = pos->end;
|
||||
bool is_string_literal = pos->type == StringLiteral;
|
||||
ASTPtr string_literal;
|
||||
|
||||
if (pos->type == Minus)
|
||||
{
|
||||
@ -866,10 +867,15 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
data_end = pos->end;
|
||||
++pos;
|
||||
}
|
||||
else if (pos->type == Number || is_string_literal)
|
||||
else if (pos->type == Number)
|
||||
{
|
||||
++pos;
|
||||
}
|
||||
else if (pos->type == StringLiteral)
|
||||
{
|
||||
if (!ParserStringLiteral().parse(begin, string_literal, expected))
|
||||
return false;
|
||||
}
|
||||
else if (isOneOf<OpeningSquareBracket, OpeningRoundBracket>(pos->type))
|
||||
{
|
||||
TokenType last_token = OpeningSquareBracket;
|
||||
@ -937,20 +943,18 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (ParserToken(DoubleColon).ignore(pos, expected)
|
||||
&& ParserDataType().parse(pos, type_ast, expected))
|
||||
{
|
||||
String s;
|
||||
size_t data_size = data_end - data_begin;
|
||||
if (is_string_literal)
|
||||
if (string_literal)
|
||||
{
|
||||
ReadBufferFromMemory buf(data_begin, data_size);
|
||||
readQuotedStringWithSQLStyle(s, buf);
|
||||
assert(buf.count() == data_size);
|
||||
node = createFunctionCast(string_literal, type_ast);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
s = String(data_begin, data_size);
|
||||
|
||||
auto literal = std::make_shared<ASTLiteral>(std::move(s));
|
||||
node = createFunctionCast(literal, type_ast);
|
||||
return true;
|
||||
{
|
||||
auto literal = std::make_shared<ASTLiteral>(String(data_begin, data_size));
|
||||
node = createFunctionCast(literal, type_ast);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
|
@ -1,4 +1,3 @@
|
||||
01624_soft_constraints
|
||||
02354_vector_search_queries
|
||||
# Check after ConstantNode refactoring
|
||||
02944_variant_as_common_type
|
||||
|
@ -94,7 +94,8 @@ class CI:
|
||||
package_type="deb",
|
||||
static_binary_name="aarch64",
|
||||
additional_pkgs=True,
|
||||
)
|
||||
),
|
||||
runner_type=Runners.BUILDER_ARM,
|
||||
),
|
||||
BuildNames.PACKAGE_ASAN: CommonJobConfigs.BUILD.with_properties(
|
||||
build_config=BuildConfig(
|
||||
|
@ -57,6 +57,7 @@ class Runners(metaclass=WithIter):
|
||||
"""
|
||||
|
||||
BUILDER = "builder"
|
||||
BUILDER_ARM = "builder-aarch64"
|
||||
STYLE_CHECKER = "style-checker"
|
||||
STYLE_CHECKER_ARM = "style-checker-aarch64"
|
||||
FUNC_TESTER = "func-tester"
|
||||
|
@ -35,10 +35,16 @@ class TestCIConfig(unittest.TestCase):
|
||||
f"Job [{job}] must have style-checker(-aarch64) runner",
|
||||
)
|
||||
elif "binary_" in job.lower() or "package_" in job.lower():
|
||||
self.assertTrue(
|
||||
CI.JOB_CONFIGS[job].runner_type == CI.Runners.BUILDER,
|
||||
f"Job [{job}] must have [{CI.Runners.BUILDER}] runner",
|
||||
)
|
||||
if job.lower() == CI.BuildNames.PACKAGE_AARCH64:
|
||||
self.assertTrue(
|
||||
CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_ARM,),
|
||||
f"Job [{job}] must have [{CI.Runners.BUILDER_ARM}] runner",
|
||||
)
|
||||
else:
|
||||
self.assertTrue(
|
||||
CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER,),
|
||||
f"Job [{job}] must have [{CI.Runners.BUILDER}] runner",
|
||||
)
|
||||
elif "aarch64" in job.lower():
|
||||
self.assertTrue(
|
||||
"aarch" in CI.JOB_CONFIGS[job].runner_type,
|
||||
|
@ -7,7 +7,8 @@
|
||||
-- sizeof(HLL) is (2^K * 6 / 8)
|
||||
-- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400
|
||||
|
||||
SET use_uncompressed_cache = 0;
|
||||
SET use_uncompressed_cache = 0;
|
||||
SET memory_profiler_step = 1;
|
||||
|
||||
-- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements
|
||||
SELECT 'UInt32';
|
||||
@ -31,14 +32,14 @@ SELECT 'K=16';
|
||||
SELECT 'UInt32';
|
||||
SET max_memory_usage = 2000000;
|
||||
SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(16)(number % 4096) u FROM numbers(4096 * 100) GROUP BY k); -- { serverError MEMORY_LIMIT_EXCEEDED }
|
||||
SET max_memory_usage = 4915200;
|
||||
SET max_memory_usage = 5230000;
|
||||
SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(16)(number % 4096) u FROM numbers(4096 * 100) GROUP BY k);
|
||||
|
||||
-- HashTable for UInt64 (used until (1<<11) elements), hence 2048 elements
|
||||
SELECT 'UInt64';
|
||||
SET max_memory_usage = 2000000;
|
||||
SELECT sum(u) FROM (SELECT intDiv(number, 2048) AS k, uniqCombined(16)(reinterpretAsString(number % 2048)) u FROM numbers(2048 * 100) GROUP BY k); -- { serverError MEMORY_LIMIT_EXCEEDED }
|
||||
SET max_memory_usage = 4915200;
|
||||
SET max_memory_usage = 5900000;
|
||||
SELECT sum(u) FROM (SELECT intDiv(number, 2048) AS k, uniqCombined(16)(reinterpretAsString(number % 2048)) u FROM numbers(2048 * 100) GROUP BY k);
|
||||
|
||||
SELECT 'K=18';
|
||||
|
@ -27,6 +27,7 @@ RENAME DICTIONARY test_01191.t TO test_01191.dict1; -- {serverError INCORRECT_QU
|
||||
DROP DICTIONARY test_01191.t; -- {serverError INCORRECT_QUERY}
|
||||
DROP TABLE test_01191.t;
|
||||
|
||||
DROP DATABASE IF EXISTS dummy_db;
|
||||
CREATE DATABASE dummy_db ENGINE=Atomic;
|
||||
RENAME DICTIONARY test_01191.dict TO dummy_db.dict1;
|
||||
RENAME DICTIONARY dummy_db.dict1 TO test_01191.dict;
|
||||
|
@ -1,2 +1,2 @@
|
||||
CREATE TABLE default.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\')
|
||||
CREATE TABLE foo.`table`\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\')
|
||||
CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\')
|
||||
CREATE TEMPORARY TABLE `table`\n(\n `key` String\n)\nENGINE = File(TSVWithNamesAndTypes, \'/dev/null\')
|
||||
|
@ -4,5 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table'
|
||||
$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table'
|
||||
$CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table'
|
||||
$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create temporary table table'
|
||||
|
@ -0,0 +1,4 @@
|
||||
414243
|
||||
ABC
|
||||
A
|
||||
{"a": \'A\'}
|
@ -0,0 +1,6 @@
|
||||
SELECT '414243'::String;
|
||||
SELECT x'414243'::String;
|
||||
SELECT b'01000001'::String;
|
||||
SELECT '{"a": \'\x41\'}'::String;
|
||||
SELECT '{"a": \'\x4\'}'::String; -- { clientError SYNTAX_ERROR }
|
||||
SELECT '{"a": \'a\x4\'}'::String; -- { clientError SYNTAX_ERROR }
|
Loading…
Reference in New Issue
Block a user