mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into clang-tidy-8
This commit is contained in:
commit
8fc59ef405
@ -176,6 +176,28 @@ function run_tests
|
||||
wait
|
||||
}
|
||||
|
||||
function get_profiles_watchdog
|
||||
{
|
||||
sleep 3000
|
||||
|
||||
echo "The trace collection did not finish in time." >> report-errors.rep
|
||||
|
||||
for pid in $(pgrep -f clickhouse)
|
||||
do
|
||||
gdb -p $pid --batch --ex "info proc all" --ex "thread apply all bt" --ex quit &> "$pid.gdb.log" &
|
||||
done
|
||||
wait
|
||||
|
||||
for i in {1..10}
|
||||
do
|
||||
if ! pkill -f clickhouse
|
||||
then
|
||||
break
|
||||
fi
|
||||
sleep 1
|
||||
done
|
||||
}
|
||||
|
||||
function get_profiles
|
||||
{
|
||||
# Collect the profiles
|
||||
@ -446,10 +468,28 @@ case "$stage" in
|
||||
time run_tests ||:
|
||||
;&
|
||||
"get_profiles")
|
||||
# Getting profiles inexplicably hangs sometimes, so try to save some logs if
|
||||
# this happens again. Give the servers 5 minutes to collect all info, then
|
||||
# trace and kill. Start in a subshell, so that both function don't interfere
|
||||
# with each other's jobs through `wait`. Also make the subshell have its own
|
||||
# process group, so that we can then kill it with all its child processes.
|
||||
# Somehow it doesn't kill the children by itself when dying.
|
||||
set -m
|
||||
( get_profiles_watchdog ) &
|
||||
watchdog_pid=$!
|
||||
set +m
|
||||
# Check that the watchdog started OK.
|
||||
kill -0 $watchdog_pid
|
||||
|
||||
# If the tests fail with OOM or something, still try to restart the servers
|
||||
# to collect the logs. Prefer not to restart, because addresses might change
|
||||
# and we won't be able to process trace_log data.
|
||||
time get_profiles || restart || get_profiles ||:
|
||||
# and we won't be able to process trace_log data. Start in a subshell, so that
|
||||
# it doesn't interfere with the watchdog through `wait`.
|
||||
( time get_profiles || restart || get_profiles ||: )
|
||||
|
||||
# Kill the whole process group, because somehow when the subshell is killed,
|
||||
# the sleep inside remains alive and orphaned.
|
||||
while env kill -- -$watchdog_pid ; do sleep 1; done
|
||||
|
||||
# Stop the servers to free memory for the subsequent query analysis.
|
||||
while killall clickhouse; do echo . ; sleep 1 ; done
|
||||
@ -465,3 +505,7 @@ case "$stage" in
|
||||
time "$script_dir/report.py" > report.html
|
||||
;&
|
||||
esac
|
||||
|
||||
# Print some final debug info to help debug Weirdness, of which there is plenty.
|
||||
jobs
|
||||
pstree -apgT
|
||||
|
@ -70,7 +70,9 @@ SelectStreamFactory::SelectStreamFactory(
|
||||
namespace
|
||||
{
|
||||
|
||||
Pipe createLocalStream(const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage, bool force_tree_shaped_pipeline)
|
||||
Pipe createLocalStream(
|
||||
const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage,
|
||||
bool add_totals_port, bool add_extremes_port, bool force_tree_shaped_pipeline)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
@ -83,12 +85,10 @@ Pipe createLocalStream(const ASTPtr & query_ast, const Block & header, const Con
|
||||
auto stream = interpreter.execute().in;
|
||||
auto source = std::make_shared<SourceFromInputStream>(std::move(stream));
|
||||
|
||||
bool add_totals_and_extremes_port = processed_stage == QueryProcessingStage::Complete;
|
||||
if (add_totals_and_extremes_port)
|
||||
{
|
||||
if (add_totals_port)
|
||||
source->addTotalsPort();
|
||||
if (add_extremes_port)
|
||||
source->addExtremesPort();
|
||||
}
|
||||
|
||||
Pipe pipe(std::move(source));
|
||||
|
||||
@ -138,7 +138,13 @@ void SelectStreamFactory::createForShard(
|
||||
Pipes & res)
|
||||
{
|
||||
bool force_add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
|
||||
bool add_totals_and_extremes_port = processed_stage == QueryProcessingStage::Complete;
|
||||
bool add_totals_port = false;
|
||||
bool add_extremes_port = false;
|
||||
if (processed_stage == QueryProcessingStage::Complete)
|
||||
{
|
||||
add_totals_port = query_ast->as<ASTSelectQuery &>().group_by_with_totals;
|
||||
add_extremes_port = context.getSettingsRef().extremes;
|
||||
}
|
||||
|
||||
auto modified_query_ast = query_ast->clone();
|
||||
if (has_virtual_shard_num_column)
|
||||
@ -146,7 +152,8 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
auto emplace_local_stream = [&]()
|
||||
{
|
||||
res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage, query_info.force_tree_shaped_pipeline));
|
||||
res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage,
|
||||
add_totals_port, add_extremes_port, query_info.force_tree_shaped_pipeline));
|
||||
};
|
||||
|
||||
String modified_query = formattedAST(modified_query_ast);
|
||||
@ -161,11 +168,10 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
auto source = std::make_shared<SourceFromInputStream>(std::move(stream), force_add_agg_info);
|
||||
|
||||
if (add_totals_and_extremes_port)
|
||||
{
|
||||
if (add_totals_port)
|
||||
source->addTotalsPort();
|
||||
if (add_extremes_port)
|
||||
source->addExtremesPort();
|
||||
}
|
||||
|
||||
res.emplace_back(std::move(source));
|
||||
};
|
||||
@ -265,7 +271,7 @@ void SelectStreamFactory::createForShard(
|
||||
auto lazily_create_stream = [
|
||||
pool = shard_info.pool, shard_num = shard_info.shard_num, modified_query, header = header, modified_query_ast, context, throttler,
|
||||
main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables,
|
||||
stage = processed_stage, local_delay]()
|
||||
stage = processed_stage, local_delay, add_totals_port, add_extremes_port]()
|
||||
-> BlockInputStreamPtr
|
||||
{
|
||||
auto current_settings = context.getSettingsRef();
|
||||
@ -298,7 +304,8 @@ void SelectStreamFactory::createForShard(
|
||||
}
|
||||
|
||||
if (try_results.empty() || local_delay < max_remote_delay)
|
||||
return std::make_shared<TreeExecutorBlockInputStream>(createLocalStream(modified_query_ast, header, context, stage, true));
|
||||
return std::make_shared<TreeExecutorBlockInputStream>(
|
||||
createLocalStream(modified_query_ast, header, context, stage, add_totals_port, add_extremes_port, true));
|
||||
else
|
||||
{
|
||||
std::vector<IConnectionPool::Entry> connections;
|
||||
@ -314,11 +321,10 @@ void SelectStreamFactory::createForShard(
|
||||
auto lazy_stream = std::make_shared<LazyBlockInputStream>("LazyShardWithLocalReplica", header, lazily_create_stream);
|
||||
auto source = std::make_shared<SourceFromInputStream>(std::move(lazy_stream), force_add_agg_info);
|
||||
|
||||
if (add_totals_and_extremes_port)
|
||||
{
|
||||
if (add_totals_port)
|
||||
source->addTotalsPort();
|
||||
if (add_extremes_port)
|
||||
source->addExtremesPort();
|
||||
}
|
||||
|
||||
res.emplace_back(std::move(source));
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <any>
|
||||
#include <limits>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
@ -647,6 +648,11 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits)
|
||||
if (overDictionary())
|
||||
throw Exception("Logical error: insert into hash-map in HashJoin over dictionary", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency.
|
||||
/// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code.
|
||||
if (unlikely(source_block.rows() > std::numeric_limits<RowRef::SizeT>::max()))
|
||||
throw Exception("Too many rows in right table block for HashJoin: " + toString(source_block.rows()), ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
/// There's no optimization for right side const columns. Remove constness if any.
|
||||
Block block = materializeBlock(source_block);
|
||||
size_t rows = block.rows();
|
||||
|
@ -1713,7 +1713,8 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
|
||||
|
||||
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
|
||||
|
||||
pipeline.dropTotalsIfHas();
|
||||
/// Forget about current totals and extremes. They will be calculated again after aggregation if needed.
|
||||
pipeline.dropTotalsAndExtremes();
|
||||
|
||||
/// If there are several sources, then we perform parallel aggregation
|
||||
if (pipeline.getNumStreams() > 1)
|
||||
|
@ -18,8 +18,10 @@ class Block;
|
||||
/// Reference to the row in block.
|
||||
struct RowRef
|
||||
{
|
||||
using SizeT = uint32_t; /// Do not use size_t cause of memory economy
|
||||
|
||||
const Block * block = nullptr;
|
||||
size_t row_num = 0;
|
||||
SizeT row_num = 0;
|
||||
|
||||
RowRef() {}
|
||||
RowRef(const Block * block_, size_t row_num_) : block(block_), row_num(row_num_) {}
|
||||
@ -33,7 +35,7 @@ struct RowRefList : RowRef
|
||||
{
|
||||
static constexpr size_t MAX_SIZE = 7; /// Adequate values are 3, 7, 15, 31.
|
||||
|
||||
size_t size = 0;
|
||||
SizeT size = 0; /// It's smaller than size_t but keeps align in Arena.
|
||||
Batch * next;
|
||||
RowRef row_refs[MAX_SIZE];
|
||||
|
||||
|
@ -401,18 +401,24 @@ void QueryPipeline::addTotals(ProcessorPtr source)
|
||||
assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline");
|
||||
|
||||
totals_having_port = &source->getOutputs().front();
|
||||
processors.emplace_back(source);
|
||||
processors.emplace_back(std::move(source));
|
||||
}
|
||||
|
||||
void QueryPipeline::dropTotalsIfHas()
|
||||
void QueryPipeline::dropTotalsAndExtremes()
|
||||
{
|
||||
if (totals_having_port)
|
||||
auto drop_port = [&](OutputPort *& port)
|
||||
{
|
||||
auto null_sink = std::make_shared<NullSink>(totals_having_port->getHeader());
|
||||
connect(*totals_having_port, null_sink->getPort());
|
||||
auto null_sink = std::make_shared<NullSink>(port->getHeader());
|
||||
connect(*port, null_sink->getPort());
|
||||
processors.emplace_back(std::move(null_sink));
|
||||
totals_having_port = nullptr;
|
||||
}
|
||||
port = nullptr;
|
||||
};
|
||||
|
||||
if (totals_having_port)
|
||||
drop_port(totals_having_port);
|
||||
|
||||
if (extremes_port)
|
||||
drop_port(extremes_port);
|
||||
}
|
||||
|
||||
void QueryPipeline::addExtremesTransform()
|
||||
|
@ -109,7 +109,8 @@ public:
|
||||
/// Add already calculated totals.
|
||||
void addTotals(ProcessorPtr source);
|
||||
|
||||
void dropTotalsIfHas();
|
||||
/// Forget about current totals and extremes. It is needed before aggregation, cause they will be calculated again.
|
||||
void dropTotalsAndExtremes();
|
||||
|
||||
/// Will read from this stream after all data was read from other streams.
|
||||
void addDelayedStream(ProcessorPtr source);
|
||||
|
@ -0,0 +1,289 @@
|
||||
20
|
||||
20
|
||||
|
||||
20
|
||||
distributed_group_by_no_merge = 0, extremes = 0
|
||||
10
|
||||
-
|
||||
10
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
------
|
||||
10
|
||||
-
|
||||
10
|
||||
-
|
||||
20
|
||||
-
|
||||
20
|
||||
------
|
||||
10
|
||||
|
||||
10
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
distributed_group_by_no_merge = 1, extremes = 0
|
||||
10
|
||||
-
|
||||
10
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
------
|
||||
10
|
||||
-
|
||||
10
|
||||
-
|
||||
20
|
||||
-
|
||||
20
|
||||
------
|
||||
10
|
||||
|
||||
10
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
distributed_group_by_no_merge = 0, extremes = 1
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
------
|
||||
|
||||
------
|
||||
------
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
------
|
||||
|
||||
------
|
||||
------
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
distributed_group_by_no_merge = 1, extremes = 1
|
||||
|
||||
distributed_group_by_no_merge = 1, extremes = 1
|
||||
distributed_group_by_no_merge = 1, extremes = 1
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
------
|
||||
|
||||
------
|
||||
------
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
------
|
||||
|
||||
------
|
||||
------
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
10
|
||||
|
||||
10
|
||||
|
||||
10
|
||||
10
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
|
||||
20
|
||||
20
|
||||
-
|
||||
|
||||
-
|
||||
-
|
||||
20
|
||||
|
||||
20
|
||||
|
||||
20
|
||||
20
|
@ -0,0 +1,106 @@
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.{1,2}', system.numbers) LIMIT 5 SETTINGS distributed_group_by_no_merge = 1);
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.{1,2}', system.numbers) LIMIT 5 SETTINGS distributed_group_by_no_merge = 1) with totals;
|
||||
|
||||
SELECT 'distributed_group_by_no_merge = 0, extremes = 0';
|
||||
SET distributed_group_by_no_merge = 0, extremes = 0;
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5)) with totals);
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5))) with totals;
|
||||
|
||||
SELECT 'distributed_group_by_no_merge = 1, extremes = 0';
|
||||
SET distributed_group_by_no_merge = 1, extremes = 0;
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5)) with totals);
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5))) with totals;
|
||||
|
||||
SELECT 'distributed_group_by_no_merge = 0, extremes = 1';
|
||||
SET distributed_group_by_no_merge = 0, extremes = 1;
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5)) with totals);
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5))) with totals;
|
||||
|
||||
SELECT 'distributed_group_by_no_merge = 1, extremes = 1';
|
||||
SET distributed_group_by_no_merge = 1, extremes = 1;
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5);
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.1', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(number) FROM (SELECT * FROM remote('127.0.0.2', system.numbers) LIMIT 5) with totals;
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5)) with totals);
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5)) with totals);
|
||||
SELECT '------';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.1', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.2', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{1,2}', numbers(5))) with totals;
|
||||
SELECT '-';
|
||||
SELECT sum(s) FROM (SELECT sum(number) as s FROM remote('127.0.0.{2,3}', numbers(5))) with totals;
|
@ -0,0 +1,9 @@
|
||||
CREATE DICTIONARY testip
|
||||
(
|
||||
`network` String,
|
||||
`test_field` String
|
||||
)
|
||||
PRIMARY KEY network
|
||||
SOURCE(FILE(PATH '/tmp/test.csv' FORMAT CSVWithNames))
|
||||
LIFETIME(MIN 0 MAX 300)
|
||||
LAYOUT(IPTRIE()); -- { serverError 137 }
|
@ -0,0 +1,31 @@
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 200 0000-00-00 00:00:00 C2
|
||||
0 0 0000-00-00 00:00:00 C3
|
||||
0 0 0000-00-00 00:00:00 C3
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 210 2020-02-16 05:22:04 N1
|
||||
0 210 2020-02-16 05:22:04 N1
|
||||
0 270 2020-02-16 05:22:04 N1
|
||||
0 270 2020-02-16 05:22:04 N1
|
||||
0 380 0000-00-00 00:00:00 N1
|
||||
0 380 0000-00-00 00:00:00 N1
|
||||
0 280 0000-00-00 00:00:00 C2
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 190 2020-02-16 05:22:05 N1
|
||||
0 160 2020-02-14 05:22:13 N1
|
||||
0 230 2020-02-14 05:22:13 N1
|
||||
0 130 2020-02-14 05:22:14 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 300 0000-00-00 00:00:00 N1
|
||||
0 0 0000-00-00 00:00:00 C2
|
||||
0 0 0000-00-00 00:00:00 N1
|
||||
0 0 0000-00-00 00:00:00 N1
|
111
tests/queries/0_stateless/01257_dictionary_mismatch_types.sql
Normal file
111
tests/queries/0_stateless/01257_dictionary_mismatch_types.sql
Normal file
File diff suppressed because one or more lines are too long
Loading…
Reference in New Issue
Block a user