Merge branch 'master' into tighten-limits-functional-tests

This commit is contained in:
Alexey Milovidov 2024-08-19 05:56:35 +02:00
commit c7a336ab98
56 changed files with 332 additions and 106 deletions

View File

@ -264,7 +264,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
if (!backslash)
{
WriteBufferFromOwnString str_buf;
formatAST(*res, str_buf, hilite, oneline || approx_query_length < max_line_length);
bool oneline_current_query = oneline || approx_query_length < max_line_length;
IAST::FormatSettings settings(str_buf, oneline_current_query, hilite);
settings.show_secrets = true;
settings.print_pretty_type_names = !oneline_current_query;
res->format(settings);
if (insert_query_payload)
{
@ -307,7 +311,11 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
else
{
WriteBufferFromOwnString str_buf;
formatAST(*res, str_buf, hilite, oneline);
bool oneline_current_query = oneline || approx_query_length < max_line_length;
IAST::FormatSettings settings(str_buf, oneline_current_query, hilite);
settings.show_secrets = true;
settings.print_pretty_type_names = !oneline_current_query;
res->format(settings);
auto res_string = str_buf.str();
WriteBufferFromOStream res_cout(std::cout, 4096);

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

@ -331,7 +331,11 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Setting
{
output_stream << std::endl;
WriteBufferFromOStream res_buf(output_stream, 4096);
formatAST(*res, res_buf);
IAST::FormatSettings format_settings(res_buf, /* one_line */ false);
format_settings.hilite = true;
format_settings.show_secrets = true;
format_settings.print_pretty_type_names = true;
res->format(format_settings);
res_buf.finalize();
output_stream << std::endl << std::endl;
}

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

@ -85,7 +85,7 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg
/// We use `formatWithPossiblyHidingSensitiveData` instead of `getColumnNameWithoutAlias` because `column->type` is an ASTFunction.
/// `getColumnNameWithoutAlias` will return name of the function with `(arguments)` even if arguments is empty.
if (column)
structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true));
structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false));
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage());
}
@ -102,7 +102,7 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error);
for (size_t i = 0; i < type_list_raw->children.size(); ++i)
structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true));
structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true, false));
}
void BaseExternalTable::initSampleBlock()

View File

@ -882,7 +882,7 @@ class IColumn;
M(Bool, use_json_alias_for_old_object_type, false, "When enabled, JSON type alias will create old experimental Object type instead of a new JSON type", 0) \
M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \
M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \
M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \
M(Bool, print_pretty_type_names, true, "Print pretty type names in the DESCRIBE query and `toTypeName` function, as well as in the `SHOW CREATE TABLE` query and the `formatQuery` function.", 0) \
M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \
M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0) \
M(SQLSecurityType, default_normal_view_sql_security, SQLSecurityType::INVOKER, "Allows to set a default value for SQL SECURITY option when creating a normal view.", 0) \

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

@ -8,7 +8,6 @@
#include <Common/quoteString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeCustom.h>

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

@ -504,7 +504,7 @@ void DatabaseOnDisk::renameTable(
}
/// It returns create table statement (even if table is detached)
/// It returns the create table statement (even if table is detached)
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const
{
ASTPtr ast;

View File

@ -43,6 +43,7 @@ public:
max_query_size = settings.max_query_size;
max_parser_depth = settings.max_parser_depth;
max_parser_backtracks = settings.max_parser_backtracks;
print_pretty_type_names = settings.print_pretty_type_names;
}
String getName() const override { return name; }
@ -138,7 +139,11 @@ private:
}
}
formatAST(*ast, buf, /*hilite*/ false, /*single_line*/ output_formatting == OutputFormatting::SingleLine);
IAST::FormatSettings settings(buf, output_formatting == OutputFormatting::SingleLine, /*hilite*/ false);
settings.show_secrets = true;
settings.print_pretty_type_names = print_pretty_type_names;
ast->format(settings);
auto formatted = buf.stringView();
const size_t res_data_new_size = res_data_size + formatted.size() + 1;
@ -165,6 +170,7 @@ private:
size_t max_query_size;
size_t max_parser_depth;
size_t max_parser_backtracks;
bool print_pretty_type_names;
};
}

View File

@ -97,7 +97,12 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl()
}
MutableColumnPtr column = ColumnString::create();
column->insert(format({.ctx = getContext(), .query = *create_query, .one_line = false}));
column->insert(format(
{
.ctx = getContext(),
.query = *create_query,
.one_line = false
}));
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{
std::move(column),

View File

@ -786,7 +786,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Verify that AST formatting is consistent:
/// If you format AST, parse it back, and format it again, you get the same string.
String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true);
String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false);
/// The query can become more verbose after formatting, so:
size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0;
@ -811,7 +811,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
chassert(ast2);
String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true);
String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false);
if (formatted1 != formatted2)
throw Exception(ErrorCodes::LOGICAL_ERROR,

View File

@ -25,7 +25,8 @@ inline String format(const SecretHidingFormatSettings & settings)
&& settings.ctx->getSettingsRef().format_display_secrets_in_show_and_select
&& settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect);
return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets);
return settings.query.formatWithPossiblyHidingSensitiveData(
settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names);
}
}

View File

@ -66,17 +66,13 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo
{
frame.need_parens = false;
/// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query.
/// We have to always backquote column names to avoid ambiguity with INDEX and other declarations in CREATE query.
format_settings.ostr << backQuote(name);
if (type)
{
format_settings.ostr << ' ';
FormatStateStacked type_frame = frame;
type_frame.indent = 0;
type->formatImpl(format_settings, state, type_frame);
type->formatImpl(format_settings, state, frame);
}
if (null_modifier)

View File

@ -40,12 +40,22 @@ void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & stat
{
settings.ostr << '(' << (settings.hilite ? hilite_none : "");
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
if (!settings.one_line && settings.print_pretty_type_names && name == "Tuple")
{
if (i != 0)
settings.ostr << ", ";
arguments->children[i]->formatImpl(settings, state, frame);
++frame.indent;
std::string indent_str = settings.one_line ? "" : "\n" + std::string(4 * frame.indent, ' ');
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
settings.ostr << ',';
settings.ostr << indent_str;
arguments->children[i]->formatImpl(settings, state, frame);
}
}
else
{
frame.expression_list_prepend_whitespace = false;
arguments->formatImpl(settings, state, frame);
}
settings.ostr << (settings.hilite ? hilite_function : "") << ')';

View File

@ -42,7 +42,8 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState
void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = "\n" + std::string(4 * (frame.indent + 1), ' ');
++frame.indent;
std::string indent_str = "\n" + std::string(4 * frame.indent, ' ');
if (frame.expression_list_prepend_whitespace)
{
@ -50,8 +51,6 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
settings.ostr << ' ';
}
++frame.indent;
for (size_t i = 0, size = children.size(); i < size; ++i)
{
if (i && separator)

View File

@ -23,12 +23,8 @@ ASTPtr ASTNameTypePair::clone() const
void ASTNameTypePair::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
settings.ostr << indent_str << backQuoteIfNeed(name) << ' ';
settings.ostr << backQuoteIfNeed(name) << ' ';
type->formatImpl(settings, state, frame);
}
}

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

@ -165,11 +165,12 @@ size_t IAST::checkDepthImpl(size_t max_depth) const
return res;
}
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const
{
WriteBufferFromOwnString buf;
FormatSettings settings(buf, one_line);
settings.show_secrets = show_secrets;
settings.print_pretty_type_names = print_pretty_type_names;
format(settings);
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
}

View File

@ -201,6 +201,7 @@ public:
bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys).
char nl_or_ws; /// Newline or whitespace.
LiteralEscapingStyle literal_escaping_style;
bool print_pretty_type_names;
explicit FormatSettings(
WriteBuffer & ostr_,
@ -209,7 +210,8 @@ public:
bool always_quote_identifiers_ = false,
IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks,
bool show_secrets_ = true,
LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular)
LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular,
bool print_pretty_type_names_ = false)
: ostr(ostr_)
, one_line(one_line_)
, hilite(hilite_)
@ -218,6 +220,7 @@ public:
, show_secrets(show_secrets_)
, nl_or_ws(one_line ? ' ' : '\n')
, literal_escaping_style(literal_escaping_style_)
, print_pretty_type_names(print_pretty_type_names_)
{
}
@ -230,6 +233,7 @@ public:
, show_secrets(other.show_secrets)
, nl_or_ws(other.nl_or_ws)
, literal_escaping_style(other.literal_escaping_style)
, print_pretty_type_names(other.print_pretty_type_names)
{
}
@ -251,7 +255,7 @@ public:
/// The state that is copied when each node is formatted. For example, nesting level.
struct FormatStateStacked
{
UInt8 indent = 0;
UInt16 indent = 0;
bool need_parens = false;
bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element.
bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required)
@ -274,7 +278,7 @@ public:
/// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
/// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const;
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const;
/** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* behaviour is due to the fact such functions are called from Client which knows nothing about
@ -283,12 +287,12 @@ public:
*/
String formatForLogging(size_t max_length = 0) const
{
return formatWithPossiblyHidingSensitiveData(max_length, true, false);
return formatWithPossiblyHidingSensitiveData(max_length, true, false, false);
}
String formatForErrorMessage() const
{
return formatWithPossiblyHidingSensitiveData(0, true, false);
return formatWithPossiblyHidingSensitiveData(0, true, false, false);
}
virtual bool hasSecretParts() const { return childrenHaveSecretParts(); }

View File

@ -395,6 +395,10 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_
part_stat.prefetch_step_marks = std::max(part_stat.prefetch_step_marks, per_part_infos[i]->min_marks_per_task);
if (part_stat.prefetch_step_marks == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
LOG_DEBUG(
log,
"Part: {}, sum_marks: {}, approx mark size: {}, prefetch_step_bytes: {}, prefetch_step_marks: {}, (ranges: {})",

View File

@ -24,6 +24,7 @@ namespace ErrorCodes
{
extern const int CANNOT_SCHEDULE_TASK;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
MergeTreeReadPool::MergeTreeReadPool(
@ -235,6 +236,10 @@ void MergeTreeReadPool::fillPerThreadInfo(size_t threads, size_t sum_marks)
const auto part_idx = current_parts.back().part_idx;
const auto min_marks_per_task = per_part_infos[part_idx]->min_marks_per_task;
if (min_marks_per_task == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
/// Do not get too few rows from part.
if (marks_in_part >= min_marks_per_task && need_marks < min_marks_per_task)
need_marks = min_marks_per_task;

View File

@ -85,6 +85,7 @@ static size_t calculateMinMarksPerTask(
min_marks_per_task = heuristic_min_marks;
}
}
LOG_TEST(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task);
return min_marks_per_task;
}

View File

@ -8,6 +8,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
@ -38,6 +39,10 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
for (const auto & info : per_part_infos)
min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task);
if (min_marks_per_task == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
extension.all_callback(
InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica));
}

View File

@ -6,6 +6,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrder(
@ -37,6 +38,10 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd
for (const auto & info : per_part_infos)
min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task);
if (min_marks_per_task == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
for (const auto & part : parts_ranges)
request.push_back({part.data_part->info, MarkRanges{}});

View File

@ -1004,6 +1004,10 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init
ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request)
{
if (request.min_number_of_marks == 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)");
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ParallelReplicasHandleRequestMicroseconds);
std::lock_guard lock(mutex);

View File

@ -7,7 +7,6 @@
#include <Interpreters/Context.h>
#include <Common/FailPoint.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/randomSeed.h>
#include <Core/ServerUUID.h>
#include <boost/algorithm/string/replace.hpp>
@ -49,6 +48,20 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage
task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); });
}
void ReplicatedMergeTreeRestartingThread::start(bool schedule)
{
LOG_TRACE(log, "Starting the restating thread, schedule: {}", schedule);
if (schedule)
task->activateAndSchedule();
else
task->activate();
}
void ReplicatedMergeTreeRestartingThread::wakeup()
{
task->schedule();
}
void ReplicatedMergeTreeRestartingThread::run()
{
if (need_stop)

View File

@ -24,16 +24,9 @@ class ReplicatedMergeTreeRestartingThread
public:
explicit ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_);
void start(bool schedule = true)
{
LOG_TRACE(log, "Starting restating thread, schedule: {}", schedule);
if (schedule)
task->activateAndSchedule();
else
task->activate();
}
void start(bool schedule);
void wakeup() { task->schedule(); }
void wakeup();
void shutdown(bool part_of_full_shutdown);

View File

@ -5194,17 +5194,16 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread)
startBeingLeader();
/// Activate replica in a separate thread if we are not calling from attach thread
restarting_thread.start(/*schedule=*/!from_attach_thread);
if (from_attach_thread)
{
LOG_TRACE(log, "Trying to startup table from right now");
/// Try activating replica in current thread.
/// Try activating replica in the current thread.
restarting_thread.run();
restarting_thread.start(false);
}
else
{
restarting_thread.start(true);
/// Wait while restarting_thread finishing initialization.
/// NOTE It does not mean that replication is actually started after receiving this event.
/// It only means that an attempt to startup replication was made.
@ -5225,7 +5224,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread)
session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]()
{
LOG_TEST(log, "Received event for expired session. Waking up restarting thread");
restarting_thread.start();
restarting_thread.start(true);
});
startBackgroundMovesIfNeeded();
@ -5294,7 +5293,6 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown()
LOG_TRACE(log, "The attach thread is shutdown");
}
restarting_thread.shutdown(/* part_of_full_shutdown */true);
/// Explicitly set the event, because the restarting thread will not set it again
startup_event.set();
@ -6342,7 +6340,7 @@ void StorageReplicatedMergeTree::alter(
"Metadata on replica is not up to date with common metadata in Zookeeper. "
"It means that this replica still not applied some of previous alters."
" Probably too many alters executing concurrently (highly not recommended). "
"You can retry this error");
"You can retry the query");
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if (query_context->getZooKeeperMetadataTransaction())

View File

@ -41,7 +41,10 @@ Pipe StorageSystemOne::read(
auto column = DataTypeUInt8().createColumnConst(1, 0u)->convertToFullColumnIfConst();
Chunk chunk({ std::move(column) }, 1);
return Pipe(std::make_shared<SourceFromSingleChunk>(std::move(header), std::move(chunk)));
auto source = std::make_shared<SourceFromSingleChunk>(std::move(header), std::move(chunk));
source->addTotalRowsApprox(1);
return Pipe(source);
}

View File

@ -59,6 +59,7 @@ MESSAGES_TO_RETRY = [
"is already started to be removing by another replica right now",
# This is from LSan, and it indicates its own internal problem:
"Unable to get registers from thread",
"You can retry",
]
MAX_RETRIES = 3

View File

@ -1,4 +1,4 @@
CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(DEFLATE_QPL),\n `data` String CODEC(DEFLATE_QPL),\n `ddd` Date CODEC(DEFLATE_QPL),\n `ddd32` Date32 CODEC(DEFLATE_QPL),\n `somenum` Float64 CODEC(DEFLATE_QPL),\n `somestr` FixedString(3) CODEC(DEFLATE_QPL),\n `othernum` Int64 CODEC(DEFLATE_QPL),\n `somearray` Array(UInt8) CODEC(DEFLATE_QPL),\n `somemap` Map(String, UInt32) CODEC(DEFLATE_QPL),\n `sometuple` Tuple(UInt16, UInt64) CODEC(DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(DEFLATE_QPL),\n `data` String CODEC(DEFLATE_QPL),\n `ddd` Date CODEC(DEFLATE_QPL),\n `ddd32` Date32 CODEC(DEFLATE_QPL),\n `somenum` Float64 CODEC(DEFLATE_QPL),\n `somestr` FixedString(3) CODEC(DEFLATE_QPL),\n `othernum` Int64 CODEC(DEFLATE_QPL),\n `somearray` Array(UInt8) CODEC(DEFLATE_QPL),\n `somemap` Map(String, UInt32) CODEC(DEFLATE_QPL),\n `sometuple` Tuple(\n UInt16,\n UInt64) CODEC(DEFLATE_QPL)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2)
2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4)
3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6)

View File

@ -1,4 +1,4 @@
CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(ZSTD_QAT(1)),\n `data` String CODEC(ZSTD_QAT(1)),\n `ddd` Date CODEC(ZSTD_QAT(1)),\n `ddd32` Date32 CODEC(ZSTD_QAT(1)),\n `somenum` Float64 CODEC(ZSTD_QAT(1)),\n `somestr` FixedString(3) CODEC(ZSTD_QAT(1)),\n `othernum` Int64 CODEC(ZSTD_QAT(1)),\n `somearray` Array(UInt8) CODEC(ZSTD_QAT(1)),\n `somemap` Map(String, UInt32) CODEC(ZSTD_QAT(1)),\n `sometuple` Tuple(UInt16, UInt64) CODEC(ZSTD_QAT(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.compression_codec\n(\n `id` UInt64 CODEC(ZSTD_QAT(1)),\n `data` String CODEC(ZSTD_QAT(1)),\n `ddd` Date CODEC(ZSTD_QAT(1)),\n `ddd32` Date32 CODEC(ZSTD_QAT(1)),\n `somenum` Float64 CODEC(ZSTD_QAT(1)),\n `somestr` FixedString(3) CODEC(ZSTD_QAT(1)),\n `othernum` Int64 CODEC(ZSTD_QAT(1)),\n `somearray` Array(UInt8) CODEC(ZSTD_QAT(1)),\n `somemap` Map(String, UInt32) CODEC(ZSTD_QAT(1)),\n `sometuple` Tuple(\n UInt16,\n UInt64) CODEC(ZSTD_QAT(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
1 hello 2018-12-14 2018-12-14 1.1 aaa 5 [1,2,3] {'k1':1,'k2':2} (1,2)
2 world 2018-12-15 2018-12-15 2.2 bbb 6 [4,5,6] {'k3':3,'k4':4} (3,4)
3 ! 2018-12-16 2018-12-16 3.3 ccc 7 [7,8,9] {'k5':5,'k6':6} (5,6)

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,12 +1,16 @@
CREATE TABLE default.tuple
(
`j` Tuple(a Int8, b String)
`j` Tuple(
a Int8,
b String)
)
ENGINE = Memory
j Tuple(\n a Int8,\n b String)
CREATE TABLE default.tuple
(
`j` Tuple(a Int8, b String)
`j` Tuple(
a Int8,
b String)
)
ENGINE = Memory
j Tuple(\n a Int8,\n b String)

View File

@ -1,20 +1,20 @@
1 1 [[1]] (1,[1])
1 1 [[1]] (1,[1])
CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192
CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192
1 1 [[1]] (1,[1])
2 2 [[2]] (2,[2])
CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192
CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192
1 1 [[1]] (1,[1])
2 2 [[2]] (2,[2])
3 3 [[3]] (3,[3])
1 1 [[1]] (1,[1])
1 1 [[1]] (1,[1])
CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192
CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192
1 1 [[1]] (1,[1])
2 2 [[2]] (2,[2])
1 1 [[1]] (1,[1])
2 2 [[2]] (2,[2])
CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192
CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(\n UInt32,\n Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192
1 1 [[1]] (1,[1])
2 2 [[2]] (2,[2])
3 3 [[3]] (3,[3])

View File

@ -7,6 +7,7 @@ ENGINE = TinyLog
CREATE TABLE test
(
`a` Int64,
`b` Tuple(a Int64)
`b` Tuple(
a Int64)
)
ENGINE = TinyLog

View File

@ -1 +1 @@
CREATE TABLE default.test_table\n(\n `col1` DateTime,\n `col2` Int64,\n `col3` AggregateFunction(1, sumMap, Tuple(Array(UInt8), Array(UInt8)))\n)\nENGINE = AggregatingMergeTree\nORDER BY (col1, col2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.test_table\n(\n `col1` DateTime,\n `col2` Int64,\n `col3` AggregateFunction(1, sumMap, Tuple(\n Array(UInt8),\n Array(UInt8)))\n)\nENGINE = AggregatingMergeTree\nORDER BY (col1, col2)\nSETTINGS index_granularity = 8192

View File

@ -510,9 +510,15 @@ CREATE TABLE system.parts
`rows_where_ttl_info.max` Array(DateTime),
`projections` Array(String),
`visible` UInt8,
`creation_tid` Tuple(UInt64, UInt64, UUID),
`creation_tid` Tuple(
UInt64,
UInt64,
UUID),
`removal_tid_lock` UInt64,
`removal_tid` Tuple(UInt64, UInt64, UUID),
`removal_tid` Tuple(
UInt64,
UInt64,
UUID),
`creation_csn` UInt64,
`removal_csn` UInt64,
`has_lightweight_delete` UInt8,

View File

@ -1,6 +1,7 @@
< X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"100000","read_bytes":"800000","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"}
< X-ClickHouse-Summary: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"}
< X-ClickHouse-Progress: {"read_rows":"100000","read_bytes":"800000","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"1","result_bytes":"272"}
< X-ClickHouse-Summary: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100001","result_rows":"1","result_bytes":"272"}

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

@ -1,4 +1,4 @@
CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(`1` Tuple(`2` Int32, `3` Int32), `4` Int32)\n)\nENGINE = Memory
CREATE TABLE default.t_tuple_numeric\n(\n `t` Tuple(\n `1` Tuple(\n `2` Int32,\n `3` Int32),\n `4` Int32)\n)\nENGINE = Memory
{"t":{"1":{"2":2,"3":3},"4":4}}
2 3 4
2 3 4

View File

@ -1,8 +1,8 @@
BACKUP_CREATED
CREATE TABLE default.test\n(\n `test` Array(Tuple(foo String, bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.test\n(\n `test` Array(Tuple(\n foo String,\n bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
BACKUP_CREATED
CREATE TABLE default.test2\n(\n `test` Nested(foo String, bar Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
RESTORED
CREATE TABLE default.test\n(\n `test` Array(Tuple(foo String, bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.test\n(\n `test` Array(Tuple(\n foo String,\n bar Float64))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
RESTORED
CREATE TABLE default.test2\n(\n `test` Nested(foo String, bar Float64)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,2 @@
0
4 4 1

View File

@ -0,0 +1,20 @@
-- Tags: no-replicated-database, no-ordinary-database
set allow_experimental_refreshable_materialized_view=1;
CREATE MATERIALIZED VIEW 03221_rmv
REFRESH AFTER 10 SECOND
(
x UInt64
)
ENGINE = Memory
AS SELECT number AS x
FROM numbers(3)
UNION ALL
SELECT rand64() AS x;
SELECT sleep(2);
SELECT read_rows, total_rows, progress FROM system.view_refreshes WHERE database = currentDatabase() and view = '03221_rmv';
DROP TABLE 03221_rmv;

View File

@ -0,0 +1,22 @@
DROP TABLE IF EXISTS test__fuzz_22 SYNC;
CREATE TABLE test__fuzz_22 (k Float32, v String) ENGINE = MergeTree ORDER BY k SETTINGS index_granularity = 1;
SYSTEM STOP MERGES test__fuzz_22;
INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1);
INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1);
INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1);
INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(1);
SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost',
merge_tree_min_rows_for_concurrent_read = 9223372036854775806, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem = 9223372036854775806;
SELECT v
FROM test__fuzz_22
ORDER BY v
LIMIT 10, 10
SETTINGS max_threads = 4
FORMAT Null; -- { serverError BAD_ARGUMENTS }
DROP TABLE test__fuzz_22 SYNC;

View File

@ -0,0 +1,56 @@
SHOW CREATE TABLE:
CREATE TABLE default.test
(
`x` Tuple(
a String,
b Array(Tuple(
c Tuple(
e String),
d String))),
`y` String
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS index_granularity = 8192
CREATE TABLE default.test
(
`x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))),
`y` String
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS index_granularity = 8192
clickhouse-format:
CREATE TABLE test
(
`x` Tuple(
a String,
b Array(Tuple(
c Tuple(
e String),
d String))),
`y` String
)
ORDER BY tuple()
CREATE TABLE test (`x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), `y` String) ORDER BY tuple()
formatQuery:
CREATE TABLE test
(
`x` Tuple(
a String,
b Array(Tuple(
c Tuple(
e String),
d String))),
`y` String
)
ORDER BY tuple()
CREATE TABLE test
(
`x` Tuple(a String, b Array(Tuple(c Tuple(e String), d String))),
`y` String
)
ORDER BY tuple()

View File

@ -0,0 +1,35 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-asan, no-msan, no-tsan
# ^ requires S3
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
echo
echo "SHOW CREATE TABLE:"
${CLICKHOUSE_CLIENT} --output-format Raw --query "
DROP TABLE IF EXISTS test;
CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ();
SET print_pretty_type_names = 1;
SHOW CREATE TABLE test;
SET print_pretty_type_names = 0;
SHOW CREATE TABLE test;
DROP TABLE test;
"
echo
echo "clickhouse-format:"
${CLICKHOUSE_FORMAT} --query "
CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ()
"
${CLICKHOUSE_FORMAT} --oneline --query "
CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ()
"
echo
echo "formatQuery:"
${CLICKHOUSE_CLIENT} --output-format Raw --query "
SELECT formatQuery('CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ()') SETTINGS print_pretty_type_names = 1;
SELECT formatQuery('CREATE TABLE test (x Tuple(a String, b Array(Tuple(c Tuple(e String), d String))), y String) ORDER BY ()') SETTINGS print_pretty_type_names = 0;
"

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 }