mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 03:25:15 +00:00
Merge branch 'master' into clickhouse-keeper
This commit is contained in:
commit
1dcbfe8739
@ -43,6 +43,9 @@ public:
|
||||
for (size_t i = 2; i < args.size() - 1; i += 2)
|
||||
dst_array_types.push_back(args[i]);
|
||||
|
||||
// Type of the ELSE branch
|
||||
dst_array_types.push_back(args.back());
|
||||
|
||||
return getLeastSupertype(dst_array_types);
|
||||
}
|
||||
|
||||
|
@ -1180,11 +1180,10 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
/// old instance of the storage. For example, AsynchronousMetrics may cause ATTACH to fail,
|
||||
/// so we allow waiting here. If database_atomic_wait_for_drop_and_detach_synchronously is disabled
|
||||
/// and old storage instance still exists it will throw exception.
|
||||
bool throw_if_table_in_use = getContext()->getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously;
|
||||
if (throw_if_table_in_use)
|
||||
database->checkDetachedTableNotInUse(create.uuid);
|
||||
else
|
||||
if (getContext()->getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously)
|
||||
database->waitDetachedTableNotInUse(create.uuid);
|
||||
else
|
||||
database->checkDetachedTableNotInUse(create.uuid);
|
||||
}
|
||||
|
||||
StoragePtr res;
|
||||
|
@ -1,16 +1,22 @@
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <stack>
|
||||
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/ArrayJoinAction.h>
|
||||
#include <stack>
|
||||
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -388,6 +394,7 @@ void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & opt
|
||||
static void explainPipelineStep(IQueryPlanStep & step, IQueryPlanStep::FormatSettings & settings)
|
||||
{
|
||||
settings.out << String(settings.offset, settings.indent_char) << "(" << step.getName() << ")\n";
|
||||
|
||||
size_t current_offset = settings.offset;
|
||||
step.describePipeline(settings);
|
||||
if (current_offset == settings.offset)
|
||||
|
@ -112,6 +112,9 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
|
||||
if (enable_parallel_reading)
|
||||
read_task_callback = context->getMergeTreeReadTaskCallback();
|
||||
|
||||
/// Add explicit description.
|
||||
setStepDescription(data.getStorageID().getFullNameNotQuoted());
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::readFromPool(
|
||||
|
@ -100,7 +100,8 @@ public:
|
||||
bool enable_parallel_reading
|
||||
);
|
||||
|
||||
String getName() const override { return "ReadFromMergeTree"; }
|
||||
static constexpr auto name = "ReadFromMergeTree";
|
||||
String getName() const override { return name; }
|
||||
|
||||
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
|
@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3)
|
||||
sleep 1
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK"
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK"
|
||||
$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK"
|
||||
|
||||
wait
|
||||
|
@ -9,7 +9,7 @@ Expression (Projection)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_order_by)
|
||||
SELECT
|
||||
timestamp,
|
||||
key
|
||||
@ -21,7 +21,7 @@ Expression (Projection)
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_order_by)
|
||||
SELECT
|
||||
timestamp,
|
||||
key
|
||||
@ -35,7 +35,7 @@ Expression (Projection)
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_order_by)
|
||||
SELECT
|
||||
timestamp,
|
||||
key
|
||||
|
@ -26,35 +26,35 @@ Expression (Projection)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_table)
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_table)
|
||||
Expression (Projection)
|
||||
Limit (preliminary LIMIT (without OFFSET))
|
||||
Sorting
|
||||
Expression (Before ORDER BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_table)
|
||||
optimize_aggregation_in_order
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Aggregating
|
||||
Expression (Before GROUP BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_table)
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Aggregating
|
||||
Expression (Before GROUP BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_table)
|
||||
Expression ((Projection + Before ORDER BY))
|
||||
Aggregating
|
||||
Expression (Before GROUP BY)
|
||||
SettingQuotaAndLimits (Set limits and quota after reading from storage)
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_table)
|
||||
second-index
|
||||
1
|
||||
1
|
||||
|
@ -1,4 +1,4 @@
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_index)
|
||||
Indexes:
|
||||
MinMax
|
||||
Keys:
|
||||
@ -32,6 +32,7 @@
|
||||
Granules: 1/2
|
||||
-----------------
|
||||
"Node Type": "ReadFromMergeTree",
|
||||
"Description": "default.test_index",
|
||||
"Indexes": [
|
||||
{
|
||||
"Type": "MinMax",
|
||||
@ -89,16 +90,16 @@
|
||||
}
|
||||
]
|
||||
-----------------
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_index)
|
||||
ReadType: InOrder
|
||||
Parts: 1
|
||||
Granules: 3
|
||||
-----------------
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.test_index)
|
||||
ReadType: InReverseOrder
|
||||
Parts: 1
|
||||
Granules: 3
|
||||
ReadFromMergeTree
|
||||
ReadFromMergeTree (default.idx)
|
||||
Indexes:
|
||||
PrimaryKey
|
||||
Keys:
|
||||
|
@ -0,0 +1,20 @@
|
||||
0 555555
|
||||
1 10
|
||||
2 555555
|
||||
3 55
|
||||
4 555555
|
||||
5 555555
|
||||
6 77
|
||||
7 555555
|
||||
8 555555
|
||||
9 95
|
||||
10 100
|
||||
11 555555
|
||||
12 555555
|
||||
13 555555
|
||||
14 555555
|
||||
15 555555
|
||||
16 555555
|
||||
17 555555
|
||||
18 555555
|
||||
19 555555
|
@ -0,0 +1,12 @@
|
||||
SELECT "number", CASE "number"
|
||||
WHEN 3 THEN 55
|
||||
WHEN 6 THEN 77
|
||||
WHEN 9 THEN 95
|
||||
ELSE CASE
|
||||
WHEN "number"=1 THEN 10
|
||||
WHEN "number"=10 THEN 100
|
||||
ELSE 555555
|
||||
END
|
||||
END AS "LONG_COL_0"
|
||||
FROM `system`.numbers
|
||||
LIMIT 20;
|
Loading…
Reference in New Issue
Block a user