Merge branch 'master' into miscellaneous-3

This commit is contained in:
Alexey Milovidov 2024-08-19 01:18:48 +02:00
commit e0dbc53b58
26 changed files with 113 additions and 73 deletions

View File

@ -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).")

View File

@ -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());
}

View File

@ -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);
}

View 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);

View File

@ -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);
}

View File

@ -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()));
}

View File

@ -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;

View File

@ -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)
{

View File

@ -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);

View File

@ -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));

View File

@ -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

View File

@ -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.

View File

@ -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,

View File

@ -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;

View File

@ -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;

View File

@ -1,4 +1,3 @@
01624_soft_constraints
02354_vector_search_queries
# Check after ConstantNode refactoring
02944_variant_as_common_type

View File

@ -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(

View File

@ -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"

View File

@ -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,

View File

@ -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';

View File

@ -1,5 +1,20 @@
-- Tags: no-fasttest
#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
session_log_query_prefix="
system flush logs;
select distinct type, user, auth_type, toString(client_address)!='::ffff:0.0.0.0' as a, client_port!=0 as b, interface from system.session_log
where user in ('default', 'nonexistsnt_user_1119', ' ', ' INTERSERVER SECRET ')
and interface in ('HTTP', 'TCP', 'TCP_Interserver')
and (user != 'default' or (a=1 and b=1)) -- FIXME: we should not write uninitialized address and port (but we do sometimes)
and event_time >= now() - interval 5 minute"
$CLICKHOUSE_CLIENT -nm -q "
select * from remote('127.0.0.2', system, one, 'default', '');
select * from remote('127.0.0.2', system, one, 'default', 'wrong password'); -- { serverError AUTHENTICATION_FAILED }
select * from remote('127.0.0.2', system, one, 'nonexistsnt_user_1119', ''); -- { serverError AUTHENTICATION_FAILED }
@ -16,9 +31,17 @@ select * from url('http://127.0.0.1:8123/?query=select+1&user=+++', LineAsString
select * from cluster('test_cluster_interserver_secret', system, one);
system flush logs;
select distinct type, user, auth_type, toString(client_address)!='::ffff:0.0.0.0' as a, client_port!=0 as b, interface from system.session_log
where user in ('default', 'nonexistsnt_user_1119', ' ', ' INTERSERVER SECRET ')
and interface in ('HTTP', 'TCP', 'TCP_Interserver')
and (user != 'default' or (a=1 and b=1)) -- FIXME: we should not write uninitialized address and port (but we do sometimes)
and event_time >= now() - interval 5 minute order by type, user, interface;
$session_log_query_prefix and type != 'Logout' order by type, user, interface;
"
# Wait for logout events.
for _ in {1..10}
do
if [ "`$CLICKHOUSE_CLIENT -q "$session_log_query_prefix and type = 'Logout'" | wc -l`" -eq 3 ]
then
break
fi
sleep 2
done
$CLICKHOUSE_CLIENT -q "$session_log_query_prefix and type = 'Logout' order by user, interface"

View File

@ -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;

View File

@ -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\')

View File

@ -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'

View File

@ -0,0 +1,4 @@
414243
ABC
A
{"a": \'A\'}

View File

@ -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 }