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

This commit is contained in:
Alexey Milovidov 2024-07-24 18:55:49 +02:00
commit 8553017e93
25 changed files with 94 additions and 121 deletions

View File

@ -7,7 +7,7 @@ keywords: [object, data type]
# Object Data Type (deprecated)
**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864).
**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-formats/json/overview) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864).
<hr />

View File

@ -49,7 +49,7 @@ enum class QueryTreeNodeType : uint8_t
/// Convert query tree node type to string
const char * toString(QueryTreeNodeType type);
/** Query tree is semantical representation of query.
/** Query tree is a semantic representation of query.
* Query tree node represent node in query tree.
* IQueryTreeNode is base class for all query tree nodes.
*

View File

@ -33,7 +33,7 @@ size_t toMilliseconds(auto duration)
return std::chrono::duration_cast<std::chrono::milliseconds>(duration).count();
}
const auto epsilon = 500us;
const auto epsilon = 1ms;
class ResolvePoolMock : public DB::HostResolver
{
@ -358,53 +358,59 @@ void check_no_failed_address(size_t iteration, auto & resolver, auto & addresses
TEST_F(ResolvePoolTest, BannedForConsiquenceFail)
{
auto history = 5ms;
auto history = 10ms;
auto resolver = make_resolver(toMilliseconds(history));
auto failed_addr = resolver->resolve();
ASSERT_TRUE(addresses.contains(*failed_addr));
auto start_at = now();
failed_addr.setFail();
auto start_at = now();
ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count));
check_no_failed_address(1, resolver, addresses, failed_addr, metrics, start_at + history - epsilon);
sleep_until(start_at + history + epsilon);
start_at = now();
resolver->update();
ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.banned_count));
failed_addr.setFail();
start_at = now();
check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + history - epsilon);
sleep_until(start_at + history + epsilon);
start_at = now();
resolver->update();
// too much time has passed
if (now() > start_at + 2*history - epsilon)
return;
ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count));
// ip still banned adter history_ms + update, because it was his second consiquent fail
check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + history - epsilon);
check_no_failed_address(2, resolver, addresses, failed_addr, metrics, start_at + 2*history - epsilon);
}
TEST_F(ResolvePoolTest, NoAditionalBannForConcurrentFail)
{
auto history = 5ms;
auto history = 10ms;
auto resolver = make_resolver(toMilliseconds(history));
auto failed_addr = resolver->resolve();
ASSERT_TRUE(addresses.contains(*failed_addr));
auto start_at = now();
failed_addr.setFail();
failed_addr.setFail();
failed_addr.setFail();
failed_addr.setFail();
failed_addr.setFail();
failed_addr.setFail();
auto start_at = now();
ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.banned_count));
@ -413,6 +419,7 @@ TEST_F(ResolvePoolTest, NoAditionalBannForConcurrentFail)
sleep_until(start_at + history + epsilon);
resolver->update();
// ip is cleared after just 1 history_ms interval.
ASSERT_EQ(3, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.banned_count));

View File

@ -383,7 +383,10 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
LockMemoryExceptionInThread::removeUniqueLock();
};
asio_opts.thread_pool_size_ = getNumberOfPhysicalCPUCores();
/// At least 16 threads for network communication in asio.
/// asio is async framework, so even with 1 thread it should be ok, but
/// still as safeguard it's better to have some redundant capacity here
asio_opts.thread_pool_size_ = std::max(16U, getNumberOfPhysicalCPUCores());
if (state_manager->isSecure())
{

View File

@ -125,23 +125,6 @@ DataTypePtr DataTypeFactory::getImpl(const String & family_name_param, const AST
{
String family_name = getAliasToOrName(family_name_param);
if (endsWith(family_name, "WithDictionary"))
{
ASTPtr low_cardinality_params = std::make_shared<ASTExpressionList>();
String param_name = family_name.substr(0, family_name.size() - strlen("WithDictionary"));
if (parameters)
{
auto func = std::make_shared<ASTFunction>();
func->name = param_name;
func->arguments = parameters;
low_cardinality_params->children.push_back(func);
}
else
low_cardinality_params->children.push_back(std::make_shared<ASTIdentifier>(param_name));
return getImpl<nullptr_on_error>("LowCardinality", low_cardinality_params);
}
const auto * creator = findCreatorByName<nullptr_on_error>(family_name);
if constexpr (nullptr_on_error)
{

View File

@ -17,13 +17,19 @@
namespace DB
{
IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const DB::ASTPtr& query_ptr_,
const DB::ContextMutablePtr& context_, const DB::SelectQueryOptions& options_)
: query_ptr(query_ptr_)
, context(context_)
, options(options_)
, max_streams(context->getSettingsRef().max_threads)
IInterpreterUnionOrSelectQuery::IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_,
const ContextMutablePtr & context_, const SelectQueryOptions & options_)
: query_ptr(query_ptr_)
, context(context_)
, options(options_)
, max_streams(context->getSettingsRef().max_threads)
{
/// FIXME All code here will work with the old analyzer, however for views over Distributed tables
/// it's possible that new analyzer will be enabled in ::getQueryProcessingStage method
/// of the underlying storage when all other parts of infrastructure are not ready for it
/// (built with old analyzer).
context->setSetting("allow_experimental_analyzer", false);
if (options.shard_num)
context->addSpecialScalar(
"_shard_num",

View File

@ -75,7 +75,6 @@
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageDummy.h>
#include <Storages/StorageMerge.h>
#include <Storages/StorageValues.h>
#include <Storages/StorageView.h>
@ -214,11 +213,11 @@ InterpreterSelectQuery::InterpreterSelectQuery(
{}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const ContextPtr & context_,
Pipe input_pipe_,
const SelectQueryOptions & options_)
: InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery())
const ASTPtr & query_ptr_,
const ContextPtr & context_,
Pipe input_pipe_,
const SelectQueryOptions & options_)
: InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery())
{}
InterpreterSelectQuery::InterpreterSelectQuery(
@ -227,18 +226,15 @@ InterpreterSelectQuery::InterpreterSelectQuery(
const StoragePtr & storage_,
const StorageMetadataPtr & metadata_snapshot_,
const SelectQueryOptions & options_)
: InterpreterSelectQuery(
query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_)
{
}
: InterpreterSelectQuery(query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_)
{}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const ContextPtr & context_,
const SelectQueryOptions & options_,
PreparedSetsPtr prepared_sets_)
: InterpreterSelectQuery(
query_ptr_, context_, std::nullopt, nullptr, options_, {}, {}, prepared_sets_)
: InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, {}, {}, prepared_sets_)
{}
InterpreterSelectQuery::~InterpreterSelectQuery() = default;

View File

@ -26,7 +26,6 @@ class Logger;
namespace DB
{
class SubqueryForSet;
class InterpreterSelectWithUnionQuery;
class Context;
class QueryPlan;

View File

@ -163,7 +163,9 @@ ReadBufferIterator::Data ReadBufferIterator::next()
{
for (const auto & object_info : read_keys)
{
if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->getFileName()))
auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->getFileName());
/// Use this format only if we have a schema reader for it.
if (format_from_file_name && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file_name))
{
format = format_from_file_name;
break;
@ -221,7 +223,9 @@ ReadBufferIterator::Data ReadBufferIterator::next()
{
for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it)
{
if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName()))
auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName());
/// Use this format only if we have a schema reader for it.
if (format_from_file_name && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file_name))
{
format = format_from_file_name;
break;

View File

@ -43,7 +43,6 @@
#include <Parsers/parseQuery.h>
#include <Parsers/IAST.h>
#include <Analyzer/Utils.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/TableNode.h>
@ -61,26 +60,20 @@
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/JoinedTables.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/Context.h>
#include <Interpreters/createBlockSelector.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/getClusterName.h>
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/getCustomKeyFilterForParallelReplicas.h>
#include <Interpreters/getHeaderForProcessingStage.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <TableFunctions/TableFunctionView.h>
#include <TableFunctions/TableFunctionFactory.h>
@ -90,7 +83,6 @@
#include <Processors/Executors/PushingPipelineExecutor.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
@ -496,7 +488,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
}
std::optional<QueryProcessingStage::Enum> optimized_stage;
if (settings.allow_experimental_analyzer)
if (query_info.query_tree)
optimized_stage = getOptimizedQueryProcessingStageAnalyzer(query_info, settings);
else
optimized_stage = getOptimizedQueryProcessingStage(query_info, settings);
@ -860,31 +852,28 @@ void StorageDistributed::read(
modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed);
modified_query_info.query_tree = std::move(query_tree_distributed);
/// Return directly (with correct header) if no shard to query.
if (modified_query_info.getCluster()->getShardsInfo().empty())
return;
}
else
{
header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
}
if (!settings.allow_experimental_analyzer)
{
modified_query_info.query = ClusterProxy::rewriteSelectQuery(
local_context, modified_query_info.query,
remote_database, remote_table, remote_table_function_ptr);
}
/// Return directly (with correct header) if no shard to query.
if (modified_query_info.getCluster()->getShardsInfo().empty())
{
if (settings.allow_experimental_analyzer)
if (modified_query_info.getCluster()->getShardsInfo().empty())
{
Pipe pipe(std::make_shared<NullSource>(header));
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
read_from_pipe->setStepDescription("Read from NullSource (Distributed)");
query_plan.addStep(std::move(read_from_pipe));
return;
Pipe pipe(std::make_shared<NullSource>(header));
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
read_from_pipe->setStepDescription("Read from NullSource (Distributed)");
query_plan.addStep(std::move(read_from_pipe));
return;
}
}
const auto & snapshot_data = assert_cast<const SnapshotData &>(*storage_snapshot->data);

View File

@ -427,7 +427,9 @@ namespace
{
for (const auto & path : paths)
{
if (auto format_from_path = FormatFactory::instance().tryGetFormatFromFileName(path))
auto format_from_path = FormatFactory::instance().tryGetFormatFromFileName(path);
/// Use this format only if we have a schema reader for it.
if (format_from_path && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_path))
{
format = format_from_path;
break;
@ -716,7 +718,9 @@ namespace
/// If format is unknown we can try to determine it by the file name.
if (!format)
{
if (auto format_from_file = FormatFactory::instance().tryGetFormatFromFileName(*filename))
auto format_from_file = FormatFactory::instance().tryGetFormatFromFileName(*filename);
/// Use this format only if we have a schema reader for it.
if (format_from_file && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file))
format = format_from_file;
}

View File

@ -737,7 +737,9 @@ namespace
{
for (const auto & url : options)
{
if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(url))
auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(url);
/// Use this format only if we have a schema reader for it.
if (format_from_file_name && FormatFactory::instance().checkIfFormatHasAnySchemaReader(*format_from_file_name))
{
format = format_from_file_name;
break;

View File

@ -172,7 +172,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType
/// with subqueries it's possible that new analyzer will be enabled in ::read method
/// of underlying storage when all other parts of infra are not ready for it
/// (built with old analyzer).
context_copy->setSetting("allow_experimental_analyzer", Field{0});
context_copy->setSetting("allow_experimental_analyzer", false);
auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns);
ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy);
auto dag = analyzer.getActionsDAG(false);

View File

@ -1,5 +1,5 @@
drop table if exists lc_dict_reading;
create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
create table lc_dict_reading (val UInt64, str LowCardinality(String), pat String) engine = MergeTree order by val SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, number) as s, s from system.numbers limit 1000000;
select sum(toUInt64(str)), sum(toUInt64(pat)) from lc_dict_reading where val < 8129 or val > 8192 * 4;
drop table if exists lc_dict_reading;

View File

@ -1,6 +1,6 @@
set allow_suspicious_low_cardinality_types = 1;
drop table if exists lc_00688;
create table lc_00688 (str StringWithDictionary, val UInt8WithDictionary) engine = MergeTree order by tuple();
create table lc_00688 (str LowCardinality(String), val LowCardinality(UInt8)) engine = MergeTree order by tuple();
insert into lc_00688 values ('a', 1), ('b', 2);
select str, str in ('a', 'd') from lc_00688;
select val, val in (1, 3) from lc_00688;

View File

@ -1,5 +1,5 @@
drop table if exists lc_prewhere;
create table lc_prewhere (key UInt64, val UInt64, str StringWithDictionary, s String) engine = MergeTree order by key settings index_granularity = 8192;
create table lc_prewhere (key UInt64, val UInt64, str LowCardinality(String), s String) engine = MergeTree order by key settings index_granularity = 8192;
insert into lc_prewhere select number, if(number < 10 or number > 8192 * 9, 1, 0), toString(number) as s, s from system.numbers limit 100000;
select sum(toUInt64(str)), sum(toUInt64(s)) from lc_prewhere prewhere val == 1;
drop table if exists lc_prewhere;

View File

@ -8,8 +8,8 @@ select 'MergeTree';
drop table if exists lc_small_dict;
drop table if exists lc_big_dict;
create table lc_small_dict (str StringWithDictionary) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
create table lc_big_dict (str StringWithDictionary) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
create table lc_small_dict (str LowCardinality(String)) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
create table lc_big_dict (str LowCardinality(String)) engine = MergeTree order by str SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
insert into lc_small_dict select toString(number % 1000) from system.numbers limit 1000000;
insert into lc_big_dict select toString(number) from system.numbers limit 1000000;

View File

@ -1,13 +1,7 @@
a
a
a
a
1
1
1
1
ab
ab
ab
ab
-

View File

@ -13,56 +13,32 @@ drop table if exists lc_null_fix_str_0;
drop table if exists lc_null_fix_str_1;
create table lc_str_0 (str LowCardinality(String)) engine = Memory;
create table lc_str_1 (str StringWithDictionary) engine = Memory;
create table lc_null_str_0 (str LowCardinality(Nullable(String))) engine = Memory;
create table lc_null_str_1 (str NullableWithDictionary(String)) engine = Memory;
create table lc_int8_0 (val LowCardinality(Int8)) engine = Memory;
create table lc_int8_1 (val Int8WithDictionary) engine = Memory;
create table lc_null_int8_0 (val LowCardinality(Nullable(Int8))) engine = Memory;
create table lc_null_int8_1 (val NullableWithDictionary(Int8)) engine = Memory;
create table lc_fix_str_0 (str LowCardinality(FixedString(2))) engine = Memory;
create table lc_fix_str_1 (str FixedStringWithDictionary(2)) engine = Memory;
create table lc_null_fix_str_0 (str LowCardinality(Nullable(FixedString(2)))) engine = Memory;
create table lc_null_fix_str_1 (str NullableWithDictionary(FixedString(2))) engine = Memory;
insert into lc_str_0 select 'a';
insert into lc_str_1 select 'a';
insert into lc_null_str_0 select 'a';
insert into lc_null_str_1 select 'a';
insert into lc_int8_0 select 1;
insert into lc_int8_1 select 1;
insert into lc_null_int8_0 select 1;
insert into lc_null_int8_1 select 1;
insert into lc_fix_str_0 select 'ab';
insert into lc_fix_str_1 select 'ab';
insert into lc_null_fix_str_0 select 'ab';
insert into lc_null_fix_str_1 select 'ab';
select str from lc_str_0;
select str from lc_str_1;
select str from lc_null_str_0;
select str from lc_null_str_1;
select val from lc_int8_0;
select val from lc_int8_1;
select val from lc_null_int8_0;
select val from lc_null_int8_1;
select str from lc_fix_str_0;
select str from lc_fix_str_1;
select str from lc_null_fix_str_0;
select str from lc_null_fix_str_1;
drop table if exists lc_str_0;
drop table if exists lc_str_1;
drop table if exists lc_null_str_0;
drop table if exists lc_null_str_1;
drop table if exists lc_int8_0;
drop table if exists lc_int8_1;
drop table if exists lc_null_int8_0;
drop table if exists lc_null_int8_1;
drop table if exists lc_fix_str_0;
drop table if exists lc_fix_str_1;
drop table if exists lc_null_fix_str_0;
drop table if exists lc_null_fix_str_1;
select '-';
SELECT toLowCardinality('a') AS s, toTypeName(s), toTypeName(length(s)) from system.one;
@ -73,7 +49,7 @@ select (toLowCardinality(z) as val) || 'b' from (select arrayJoin(['c', 'd']) a
select '-';
drop table if exists lc_str_uuid;
create table lc_str_uuid(str1 String, str2 LowCardinality(String), str3 StringWithDictionary) ENGINE=Memory;
create table lc_str_uuid(str1 String, str2 LowCardinality(String), str3 LowCardinality(String)) ENGINE=Memory;
select toUUID(str1), toUUID(str2), toUUID(str3) from lc_str_uuid;
select toUUID(str1, '', NULL), toUUID(str2, '', NULL), toUUID(str3, '', NULL) from lc_str_uuid;
insert into lc_str_uuid values ('61f0c404-5cb3-11e7-907b-a6006ad3dba0', '61f0c404-5cb3-11e7-907b-a6006ad3dba0', '61f0c404-5cb3-11e7-907b-a6006ad3dba0');

View File

@ -1,5 +1,5 @@
drop table if exists tab_00717;
create table tab_00717 (a String, b StringWithDictionary) engine = MergeTree order by a;
create table tab_00717 (a String, b LowCardinality(String)) engine = MergeTree order by a;
insert into tab_00717 values ('a_1', 'b_1'), ('a_2', 'b_2');
select count() from tab_00717;
select a from tab_00717 group by a order by a;

View File

@ -7,7 +7,7 @@ alter table tab_00718 modify column b UInt32;
select *, toTypeName(b) from tab_00718;
alter table tab_00718 modify column b LowCardinality(UInt32);
select *, toTypeName(b) from tab_00718;
alter table tab_00718 modify column b StringWithDictionary;
alter table tab_00718 modify column b LowCardinality(String);
select *, toTypeName(b) from tab_00718;
alter table tab_00718 modify column b LowCardinality(UInt32);
select *, toTypeName(b) from tab_00718;

View File

@ -1,7 +1,7 @@
drop table if exists lc_00752;
drop table if exists lc_mv_00752;
create table lc_00752 (str StringWithDictionary) engine = MergeTree order by tuple();
create table lc_00752 (str LowCardinality(String)) engine = MergeTree order by tuple();
insert into lc_00752 values ('a'), ('bbb'), ('ab'), ('accccc'), ('baasddas'), ('bcde');
@ -12,4 +12,3 @@ select * from lc_mv_00752 order by letter;
drop table if exists lc_00752;
drop table if exists lc_mv_00752;

View File

@ -28,7 +28,7 @@ ORDER BY tuple();
INSERT INTO t_01411_num (num) SELECT number % 1000 FROM numbers(100000);
create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val;
create table lc_dict_reading (val UInt64, str LowCardinality(String), pat String) engine = MergeTree order by val;
insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, number) as s, s from system.numbers limit 100000;
"""

View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
touch $CLICKHOUSE_TEST_UNIQUE_NAME.xml
$CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.*')" 2>&1 | grep -c "CANNOT_DETECT_FORMAT"
rm $CLICKHOUSE_TEST_UNIQUE_NAME.xml