mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge branch 'master' into turn_on_s3_tests
This commit is contained in:
commit
55913cf8e1
10
docs/en/interfaces/third-party/gui.md
vendored
10
docs/en/interfaces/third-party/gui.md
vendored
@ -147,6 +147,16 @@ Features:
|
||||
|
||||
[Zeppelin-Interpreter-for-ClickHouse](https://github.com/SiderZhang/Zeppelin-Interpreter-for-ClickHouse) is a [Zeppelin](https://zeppelin.apache.org) interpreter for ClickHouse. Compared with JDBC interpreter, it can provide better timeout control for long running queries.
|
||||
|
||||
### ClickCat {#clickcat}
|
||||
|
||||
[ClickCat](https://github.com/open-botech/ClickCat) is a firendly user interface that lets you search, explore and visualize your ClickHouse Data.
|
||||
|
||||
Features:
|
||||
|
||||
- An online SQL editor which can run your SQL code without any installing.
|
||||
- You can observe all processes and mutations. For those unfinished processes, you can kill them in ui.
|
||||
- The Metrics contains Cluster Analysis,Data Analysis,Query Analysis.
|
||||
|
||||
## Commercial {#commercial}
|
||||
|
||||
### DataGrip {#datagrip}
|
||||
|
@ -638,10 +638,6 @@ void LRUFileCache::remove(const Key & key)
|
||||
|
||||
if (fs::exists(key_path))
|
||||
fs::remove(key_path);
|
||||
|
||||
#ifndef NDEBUG
|
||||
assertCacheCorrectness(cache_lock);
|
||||
#endif
|
||||
}
|
||||
|
||||
void LRUFileCache::remove()
|
||||
@ -1053,8 +1049,7 @@ void LRUFileCache::assertCacheCellsCorrectness(
|
||||
if (file_segment->reserved_size != 0)
|
||||
{
|
||||
assert(cell.queue_iterator);
|
||||
/// FIXME: this is too slow, need to make it O(1)
|
||||
/// assert(queue.contains(file_segment->key(), file_segment->offset(), cache_lock));
|
||||
assert(queue.contains(file_segment->key(), file_segment->offset(), cache_lock));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -301,7 +301,7 @@ private:
|
||||
|
||||
size_t getFileSegmentsNumUnlocked(std::lock_guard<std::mutex> & cache_lock) const;
|
||||
|
||||
static void assertCacheCellsCorrectness(const FileSegmentsByOffset & cells_by_offset, std::lock_guard<std::mutex> & cache_lock);
|
||||
void assertCacheCellsCorrectness(const FileSegmentsByOffset & cells_by_offset, std::lock_guard<std::mutex> & cache_lock);
|
||||
|
||||
public:
|
||||
String dumpStructure(const Key & key_) override;
|
||||
|
@ -1166,12 +1166,12 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina
|
||||
? list_watches
|
||||
: watches;
|
||||
|
||||
watches_type[zk_request->getPath()].emplace_back(session_id);
|
||||
watches_type[zk_request->getPath()].emplace(session_id);
|
||||
sessions_and_watchers[session_id].emplace(zk_request->getPath());
|
||||
}
|
||||
else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists)
|
||||
{
|
||||
watches[zk_request->getPath()].emplace_back(session_id);
|
||||
watches[zk_request->getPath()].emplace(session_id);
|
||||
sessions_and_watchers[session_id].emplace(zk_request->getPath());
|
||||
}
|
||||
}
|
||||
@ -1206,13 +1206,7 @@ void KeeperStorage::clearDeadWatches(int64_t session_id)
|
||||
if (watch != watches.end())
|
||||
{
|
||||
auto & watches_for_path = watch->second;
|
||||
for (auto w_it = watches_for_path.begin(); w_it != watches_for_path.end();)
|
||||
{
|
||||
if (*w_it == session_id)
|
||||
w_it = watches_for_path.erase(w_it);
|
||||
else
|
||||
++w_it;
|
||||
}
|
||||
watches_for_path.erase(session_id);
|
||||
if (watches_for_path.empty())
|
||||
watches.erase(watch);
|
||||
}
|
||||
@ -1222,13 +1216,7 @@ void KeeperStorage::clearDeadWatches(int64_t session_id)
|
||||
if (list_watch != list_watches.end())
|
||||
{
|
||||
auto & list_watches_for_path = list_watch->second;
|
||||
for (auto w_it = list_watches_for_path.begin(); w_it != list_watches_for_path.end();)
|
||||
{
|
||||
if (*w_it == session_id)
|
||||
w_it = list_watches_for_path.erase(w_it);
|
||||
else
|
||||
++w_it;
|
||||
}
|
||||
list_watches_for_path.erase(session_id);
|
||||
if (list_watches_for_path.empty())
|
||||
list_watches.erase(list_watch);
|
||||
}
|
||||
@ -1250,7 +1238,7 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const
|
||||
|
||||
void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
auto write_int_vec = [&buf](const std::vector<int64_t> & session_ids)
|
||||
auto write_int_container = [&buf](const auto & session_ids)
|
||||
{
|
||||
for (int64_t session_id : session_ids)
|
||||
{
|
||||
@ -1261,13 +1249,13 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
for (const auto & [watch_path, sessions] : watches)
|
||||
{
|
||||
buf << watch_path << "\n";
|
||||
write_int_vec(sessions);
|
||||
write_int_container(sessions);
|
||||
}
|
||||
|
||||
for (const auto & [watch_path, sessions] : list_watches)
|
||||
{
|
||||
buf << watch_path << "\n";
|
||||
write_int_vec(sessions);
|
||||
write_int_container(sessions);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -96,7 +96,7 @@ public:
|
||||
using Container = SnapshotableHashTable<Node>;
|
||||
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionIDs = std::vector<int64_t>;
|
||||
using SessionIDs = std::unordered_set<int64_t>;
|
||||
|
||||
/// Just vector of SHA1 from user:password
|
||||
using AuthIDs = std::vector<AuthID>;
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_set>
|
||||
#include <vector>
|
||||
|
||||
|
||||
@ -7,6 +8,8 @@ namespace DB
|
||||
{
|
||||
|
||||
using ColumnNumbers = std::vector<size_t>;
|
||||
using ColumnNumbersSet = std::unordered_set<size_t>;
|
||||
using ColumnNumbersList = std::vector<ColumnNumbers>;
|
||||
using ColumnNumbersSetList = std::vector<ColumnNumbersSet>;
|
||||
|
||||
}
|
||||
|
@ -16,6 +16,7 @@ using NameOrderedSet = std::set<std::string>;
|
||||
using NameToNameMap = std::unordered_map<std::string, std::string>;
|
||||
using NameToNameSetMap = std::unordered_map<std::string, NameSet>;
|
||||
using NameToNameVector = std::vector<std::pair<std::string, std::string>>;
|
||||
using NameToIndexMap = std::unordered_map<std::string, size_t>;
|
||||
|
||||
using NameWithAlias = std::pair<std::string, std::string>;
|
||||
using NamesWithAliases = std::vector<NameWithAlias>;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <cstddef>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
#include <base/sort.h>
|
||||
@ -214,4 +215,17 @@ std::optional<NameAndTypePair> NamesAndTypesList::tryGetByName(const std::string
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
size_t NamesAndTypesList::getPosByName(const std::string &name) const noexcept
|
||||
{
|
||||
size_t pos = 0;
|
||||
for (const NameAndTypePair & column : *this)
|
||||
{
|
||||
if (column.name == name)
|
||||
break;
|
||||
++pos;
|
||||
}
|
||||
return pos;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -105,8 +105,11 @@ public:
|
||||
/// Check that column contains in list
|
||||
bool contains(const String & name) const;
|
||||
|
||||
/// Try to get column by name, return empty optional if column not found
|
||||
/// Try to get column by name, returns empty optional if column not found
|
||||
std::optional<NameAndTypePair> tryGetByName(const std::string & name) const;
|
||||
|
||||
/// Try to get column position by name, returns number of columns if column isn't found
|
||||
size_t getPosByName(const std::string & name) const noexcept;
|
||||
};
|
||||
|
||||
using NamesAndTypesLists = std::vector<NamesAndTypesList>;
|
||||
|
@ -567,7 +567,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
\
|
||||
M(UInt64, remote_fs_read_max_backoff_ms, 10000, "Max wait time when trying to read data for remote disk", 0) \
|
||||
M(UInt64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \
|
||||
M(Bool, enable_filesystem_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \
|
||||
M(Bool, enable_filesystem_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must be done via disk config), but allows to bypass cache for some queries if intended", 0) \
|
||||
M(UInt64, filesystem_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \
|
||||
M(Bool, enable_filesystem_cache_on_write_operations, false, "Write into cache on write operations. To actually work this setting requires be added to disk config too", 0) \
|
||||
M(Bool, enable_filesystem_cache_log, false, "Allows to record the filesystem caching log for each query", 0) \
|
||||
|
@ -584,6 +584,13 @@ bool FormatFactory::checkIfFormatHasAnySchemaReader(const String & name) const
|
||||
return checkIfFormatHasSchemaReader(name) || checkIfFormatHasExternalSchemaReader(name);
|
||||
}
|
||||
|
||||
void FormatFactory::checkFormatName(const String & name) const
|
||||
{
|
||||
auto it = dict.find(name);
|
||||
if (it == dict.end())
|
||||
throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
|
||||
FormatFactory & FormatFactory::instance()
|
||||
{
|
||||
static FormatFactory ret;
|
||||
|
@ -210,6 +210,9 @@ public:
|
||||
bool isInputFormat(const String & name) const;
|
||||
bool isOutputFormat(const String & name) const;
|
||||
|
||||
/// Check that format with specified name exists and throw an exception otherwise.
|
||||
void checkFormatName(const String & name) const;
|
||||
|
||||
private:
|
||||
FormatsDictionary dict;
|
||||
FileExtensionFormats file_extension_formats;
|
||||
|
165
src/Functions/grouping.h
Normal file
165
src/Functions/grouping.h
Normal file
@ -0,0 +1,165 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionGroupingBase : public IFunction
|
||||
{
|
||||
protected:
|
||||
static constexpr UInt64 ONE = 1;
|
||||
|
||||
const ColumnNumbers arguments_indexes;
|
||||
|
||||
public:
|
||||
FunctionGroupingBase(ColumnNumbers arguments_indexes_)
|
||||
: arguments_indexes(std::move(arguments_indexes_))
|
||||
{}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
bool isSuitableForConstantFolding() const override { return false; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||
{
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
template <typename AggregationKeyChecker>
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, size_t input_rows_count, AggregationKeyChecker checker) const
|
||||
{
|
||||
const auto * grouping_set_column = checkAndGetColumn<ColumnUInt64>(arguments[0].column.get());
|
||||
|
||||
auto result = ColumnUInt64::create();
|
||||
auto & result_data = result->getData();
|
||||
result_data.reserve(input_rows_count);
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
UInt64 set_index = grouping_set_column->getElement(i);
|
||||
|
||||
UInt64 value = 0;
|
||||
for (auto index : arguments_indexes)
|
||||
value = (value << 1) + (checker(set_index, index) ? 1 : 0);
|
||||
|
||||
result_data.push_back(value);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionGroupingOrdinary : public FunctionGroupingBase
|
||||
{
|
||||
public:
|
||||
explicit FunctionGroupingOrdinary(ColumnNumbers arguments_indexes_)
|
||||
: FunctionGroupingBase(std::move(arguments_indexes_))
|
||||
{}
|
||||
|
||||
String getName() const override { return "groupingOrdinary"; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
UInt64 value = (ONE << arguments_indexes.size()) - 1;
|
||||
return ColumnUInt64::create(input_rows_count, value);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionGroupingForRollup : public FunctionGroupingBase
|
||||
{
|
||||
const UInt64 aggregation_keys_number;
|
||||
|
||||
public:
|
||||
FunctionGroupingForRollup(ColumnNumbers arguments_indexes_, UInt64 aggregation_keys_number_)
|
||||
: FunctionGroupingBase(std::move(arguments_indexes_))
|
||||
, aggregation_keys_number(aggregation_keys_number_)
|
||||
{}
|
||||
|
||||
String getName() const override { return "groupingForRollup"; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
return FunctionGroupingBase::executeImpl(arguments, input_rows_count,
|
||||
[this](UInt64 set_index, UInt64 arg_index)
|
||||
{
|
||||
// For ROLLUP(a, b, c) there will be following grouping set indexes:
|
||||
// | GROUPING SET | INDEX |
|
||||
// | (a, b, c) | 0 |
|
||||
// | (a, b) | 1 |
|
||||
// | (a) | 2 |
|
||||
// | () | 3 |
|
||||
return arg_index < aggregation_keys_number - set_index;
|
||||
}
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionGroupingForCube : public FunctionGroupingBase
|
||||
{
|
||||
const UInt64 aggregation_keys_number;
|
||||
|
||||
public:
|
||||
|
||||
FunctionGroupingForCube(ColumnNumbers arguments_indexes_, UInt64 aggregation_keys_number_)
|
||||
: FunctionGroupingBase(arguments_indexes_)
|
||||
, aggregation_keys_number(aggregation_keys_number_)
|
||||
{}
|
||||
|
||||
String getName() const override { return "groupingForCube"; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
return FunctionGroupingBase::executeImpl(arguments, input_rows_count,
|
||||
[this](UInt64 set_index, UInt64 arg_index)
|
||||
{
|
||||
// For CUBE(a, b) there will be following grouping set indexes:
|
||||
// | GROUPING SET | INDEX |
|
||||
// | (a, b) | 0 |
|
||||
// | (a) | 1 |
|
||||
// | (b) | 2 |
|
||||
// | () | 3 |
|
||||
auto set_mask = (ONE << aggregation_keys_number) - 1 - set_index;
|
||||
return set_mask & (ONE << (aggregation_keys_number - arg_index - 1));
|
||||
}
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionGroupingForGroupingSets : public FunctionGroupingBase
|
||||
{
|
||||
ColumnNumbersSetList grouping_sets;
|
||||
public:
|
||||
FunctionGroupingForGroupingSets(ColumnNumbers arguments_indexes_, ColumnNumbersList const & grouping_sets_)
|
||||
: FunctionGroupingBase(std::move(arguments_indexes_))
|
||||
{
|
||||
for (auto const & set : grouping_sets_)
|
||||
grouping_sets.emplace_back(set.begin(), set.end());
|
||||
}
|
||||
|
||||
String getName() const override { return "groupingForGroupingSets"; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
return FunctionGroupingBase::executeImpl(arguments, input_rows_count,
|
||||
[this](UInt64 set_index, UInt64 arg_index)
|
||||
{
|
||||
return grouping_sets[set_index].contains(arg_index);
|
||||
}
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,12 @@
|
||||
#include <memory>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
|
||||
#include <Functions/grouping.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsMiscellaneous.h>
|
||||
|
||||
@ -8,10 +14,12 @@
|
||||
|
||||
#include <DataTypes/DataTypeSet.h>
|
||||
#include <DataTypes/DataTypeFunction.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
|
||||
#include <Columns/ColumnSet.h>
|
||||
@ -56,6 +64,9 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_ELEMENT_OF_SET;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
||||
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
|
||||
}
|
||||
|
||||
static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols)
|
||||
@ -459,10 +470,18 @@ public:
|
||||
};
|
||||
|
||||
ActionsMatcher::Data::Data(
|
||||
ContextPtr context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, ActionsDAGPtr actions_dag,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_)
|
||||
ContextPtr context_,
|
||||
SizeLimits set_size_limit_,
|
||||
size_t subquery_depth_,
|
||||
std::reference_wrapper<const NamesAndTypesList> source_columns_,
|
||||
ActionsDAGPtr actions_dag,
|
||||
PreparedSets & prepared_sets_,
|
||||
SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_,
|
||||
bool no_makeset_,
|
||||
bool only_consts_,
|
||||
bool create_source_for_in_,
|
||||
AggregationKeysInfo aggregation_keys_info_)
|
||||
: WithContext(context_)
|
||||
, set_size_limit(set_size_limit_)
|
||||
, subquery_depth(subquery_depth_)
|
||||
@ -475,6 +494,7 @@ ActionsMatcher::Data::Data(
|
||||
, create_source_for_in(create_source_for_in_)
|
||||
, visit_depth(0)
|
||||
, actions_stack(std::move(actions_dag), context_)
|
||||
, aggregation_keys_info(aggregation_keys_info_)
|
||||
, next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1)
|
||||
{
|
||||
}
|
||||
@ -817,6 +837,52 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
return;
|
||||
}
|
||||
|
||||
if (node.name == "grouping")
|
||||
{
|
||||
size_t arguments_size = node.arguments->children.size();
|
||||
if (arguments_size == 0)
|
||||
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function GROUPING expects at least one argument");
|
||||
if (arguments_size > 64)
|
||||
throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Function GROUPING can have up to 64 arguments, but {} provided", arguments_size);
|
||||
auto keys_info = data.aggregation_keys_info;
|
||||
auto aggregation_keys_number = keys_info.aggregation_keys.size();
|
||||
|
||||
ColumnNumbers arguments_indexes;
|
||||
for (auto const & arg : node.arguments->children)
|
||||
{
|
||||
size_t pos = keys_info.aggregation_keys.getPosByName(arg->getColumnName());
|
||||
if (pos == aggregation_keys_number)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of GROUPING function {} is not a part of GROUP BY clause", arg->getColumnName());
|
||||
arguments_indexes.push_back(pos);
|
||||
}
|
||||
|
||||
switch (keys_info.group_by_kind)
|
||||
{
|
||||
case GroupByKind::GROUPING_SETS:
|
||||
{
|
||||
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingForGroupingSets>(std::move(arguments_indexes), keys_info.grouping_set_keys)), { "__grouping_set" }, column_name);
|
||||
break;
|
||||
}
|
||||
case GroupByKind::ROLLUP:
|
||||
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingForRollup>(std::move(arguments_indexes), aggregation_keys_number)), { "__grouping_set" }, column_name);
|
||||
break;
|
||||
case GroupByKind::CUBE:
|
||||
{
|
||||
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingForCube>(std::move(arguments_indexes), aggregation_keys_number)), { "__grouping_set" }, column_name);
|
||||
break;
|
||||
}
|
||||
case GroupByKind::ORDINARY:
|
||||
{
|
||||
data.addFunction(std::make_shared<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGroupingOrdinary>(std::move(arguments_indexes))), {}, column_name);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Unexpected kind of GROUP BY clause for GROUPING function: {}", keys_info.group_by_kind);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
SetPtr prepared_set;
|
||||
if (checkFunctionIsInOrGlobalInOperator(node))
|
||||
{
|
||||
|
@ -1,10 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Interpreters/SubqueryForSet.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -76,6 +78,42 @@ class ASTIdentifier;
|
||||
class ASTFunction;
|
||||
class ASTLiteral;
|
||||
|
||||
enum class GroupByKind
|
||||
{
|
||||
NONE,
|
||||
ORDINARY,
|
||||
ROLLUP,
|
||||
CUBE,
|
||||
GROUPING_SETS,
|
||||
};
|
||||
|
||||
/*
|
||||
* This class stores information about aggregation keys used in GROUP BY clause.
|
||||
* It's used for providing information about aggregation to GROUPING function
|
||||
* implementation.
|
||||
*/
|
||||
struct AggregationKeysInfo
|
||||
{
|
||||
AggregationKeysInfo(
|
||||
std::reference_wrapper<const NamesAndTypesList> aggregation_keys_,
|
||||
std::reference_wrapper<const ColumnNumbersList> grouping_set_keys_,
|
||||
GroupByKind group_by_kind_)
|
||||
: aggregation_keys(aggregation_keys_)
|
||||
, grouping_set_keys(grouping_set_keys_)
|
||||
, group_by_kind(group_by_kind_)
|
||||
{}
|
||||
|
||||
AggregationKeysInfo(const AggregationKeysInfo &) = default;
|
||||
AggregationKeysInfo(AggregationKeysInfo &&) = default;
|
||||
|
||||
// Names and types of all used keys
|
||||
const NamesAndTypesList & aggregation_keys;
|
||||
// Indexes of aggregation keys used in each grouping set (only for GROUP BY GROUPING SETS)
|
||||
const ColumnNumbersList & grouping_set_keys;
|
||||
|
||||
GroupByKind group_by_kind;
|
||||
};
|
||||
|
||||
/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too.
|
||||
class ActionsMatcher
|
||||
{
|
||||
@ -95,6 +133,7 @@ public:
|
||||
bool create_source_for_in;
|
||||
size_t visit_depth;
|
||||
ScopeStack actions_stack;
|
||||
AggregationKeysInfo aggregation_keys_info;
|
||||
|
||||
/*
|
||||
* Remember the last unique column suffix to avoid quadratic behavior
|
||||
@ -107,14 +146,15 @@ public:
|
||||
ContextPtr context_,
|
||||
SizeLimits set_size_limit_,
|
||||
size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_,
|
||||
std::reference_wrapper<const NamesAndTypesList> source_columns_,
|
||||
ActionsDAGPtr actions_dag,
|
||||
PreparedSets & prepared_sets_,
|
||||
SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_,
|
||||
bool no_makeset_,
|
||||
bool only_consts_,
|
||||
bool create_source_for_in_);
|
||||
bool create_source_for_in_,
|
||||
AggregationKeysInfo aggregation_keys_info_);
|
||||
|
||||
/// Does result of the calculation already exists in the block.
|
||||
bool hasColumn(const String & column_name) const;
|
||||
|
@ -43,10 +43,13 @@
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/GetAggregatesVisitor.h>
|
||||
@ -325,12 +328,21 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
|
||||
{
|
||||
if (ASTPtr group_by_ast = select_query->groupBy())
|
||||
{
|
||||
NameSet unique_keys;
|
||||
NameToIndexMap unique_keys;
|
||||
ASTs & group_asts = group_by_ast->children;
|
||||
|
||||
if (select_query->group_by_with_rollup)
|
||||
group_by_kind = GroupByKind::ROLLUP;
|
||||
else if (select_query->group_by_with_cube)
|
||||
group_by_kind = GroupByKind::CUBE;
|
||||
else if (select_query->group_by_with_grouping_sets && group_asts.size() > 1)
|
||||
group_by_kind = GroupByKind::GROUPING_SETS;
|
||||
else
|
||||
group_by_kind = GroupByKind::ORDINARY;
|
||||
|
||||
/// For GROUPING SETS with multiple groups we always add virtual __grouping_set column
|
||||
/// With set number, which is used as an additional key at the stage of merging aggregating data.
|
||||
if (select_query->group_by_with_grouping_sets && group_asts.size() > 1)
|
||||
if (group_by_kind != GroupByKind::ORDINARY)
|
||||
aggregated_columns.emplace_back("__grouping_set", std::make_shared<DataTypeUInt64>());
|
||||
|
||||
for (ssize_t i = 0; i < static_cast<ssize_t>(group_asts.size()); ++i)
|
||||
@ -347,6 +359,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
|
||||
group_elements_ast = group_ast_element->children;
|
||||
|
||||
NamesAndTypesList grouping_set_list;
|
||||
ColumnNumbers grouping_set_indexes_list;
|
||||
|
||||
for (ssize_t j = 0; j < ssize_t(group_elements_ast.size()); ++j)
|
||||
{
|
||||
@ -387,15 +400,21 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
|
||||
/// Aggregation keys are unique.
|
||||
if (!unique_keys.contains(key.name))
|
||||
{
|
||||
unique_keys.insert(key.name);
|
||||
unique_keys[key.name] = aggregation_keys.size();
|
||||
grouping_set_indexes_list.push_back(aggregation_keys.size());
|
||||
aggregation_keys.push_back(key);
|
||||
|
||||
/// Key is no longer needed, therefore we can save a little by moving it.
|
||||
aggregated_columns.push_back(std::move(key));
|
||||
}
|
||||
else
|
||||
{
|
||||
grouping_set_indexes_list.push_back(unique_keys[key.name]);
|
||||
}
|
||||
}
|
||||
|
||||
aggregation_keys_list.push_back(std::move(grouping_set_list));
|
||||
aggregation_keys_indexes_list.push_back(std::move(grouping_set_indexes_list));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -433,7 +452,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
|
||||
/// Aggregation keys are uniqued.
|
||||
if (!unique_keys.contains(key.name))
|
||||
{
|
||||
unique_keys.insert(key.name);
|
||||
unique_keys[key.name] = aggregation_keys.size();
|
||||
aggregation_keys.push_back(key);
|
||||
|
||||
/// Key is no longer needed, therefore we can save a little by moving it.
|
||||
@ -442,6 +461,13 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
|
||||
}
|
||||
}
|
||||
|
||||
if (!select_query->group_by_with_grouping_sets)
|
||||
{
|
||||
auto & list = aggregation_keys_indexes_list.emplace_back();
|
||||
for (size_t i = 0; i < aggregation_keys.size(); ++i)
|
||||
list.push_back(i);
|
||||
}
|
||||
|
||||
if (group_asts.empty())
|
||||
{
|
||||
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {});
|
||||
@ -583,7 +609,8 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_
|
||||
no_makeset_for_subqueries,
|
||||
false /* no_makeset */,
|
||||
only_consts,
|
||||
!isRemoteStorage() /* create_source_for_in */);
|
||||
!isRemoteStorage() /* create_source_for_in */,
|
||||
getAggregationKeysInfo());
|
||||
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
||||
actions = visitor_data.getActions();
|
||||
}
|
||||
@ -603,7 +630,8 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGP
|
||||
true /* no_makeset_for_subqueries, no_makeset implies no_makeset_for_subqueries */,
|
||||
true /* no_makeset */,
|
||||
only_consts,
|
||||
!isRemoteStorage() /* create_source_for_in */);
|
||||
!isRemoteStorage() /* create_source_for_in */,
|
||||
getAggregationKeysInfo());
|
||||
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
||||
actions = visitor_data.getActions();
|
||||
}
|
||||
@ -624,7 +652,8 @@ void ExpressionAnalyzer::getRootActionsForHaving(
|
||||
no_makeset_for_subqueries,
|
||||
false /* no_makeset */,
|
||||
only_consts,
|
||||
true /* create_source_for_in */);
|
||||
true /* create_source_for_in */,
|
||||
getAggregationKeysInfo());
|
||||
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
||||
actions = visitor_data.getActions();
|
||||
}
|
||||
|
@ -1,6 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/SubqueryForSet.h>
|
||||
@ -67,6 +69,7 @@ struct ExpressionAnalyzerData
|
||||
bool has_aggregation = false;
|
||||
NamesAndTypesList aggregation_keys;
|
||||
NamesAndTypesLists aggregation_keys_list;
|
||||
ColumnNumbersList aggregation_keys_indexes_list;
|
||||
bool has_const_aggregation_keys = false;
|
||||
AggregateDescriptions aggregate_descriptions;
|
||||
|
||||
@ -77,6 +80,8 @@ struct ExpressionAnalyzerData
|
||||
|
||||
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
|
||||
TemporaryTablesMapping external_tables;
|
||||
|
||||
GroupByKind group_by_kind = GroupByKind::NONE;
|
||||
};
|
||||
|
||||
|
||||
@ -200,6 +205,11 @@ protected:
|
||||
|
||||
NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
|
||||
NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
|
||||
|
||||
AggregationKeysInfo getAggregationKeysInfo() const noexcept
|
||||
{
|
||||
return { aggregation_keys, aggregation_keys_indexes_list, group_by_kind };
|
||||
}
|
||||
};
|
||||
|
||||
class SelectQueryExpressionAnalyzer;
|
||||
|
@ -1098,6 +1098,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
if (query.group_by_with_grouping_sets && (query.group_by_with_rollup || query.group_by_with_cube))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUPING SETS are not supported together with ROLLUP and CUBE");
|
||||
|
||||
if (expressions.hasHaving() && query.group_by_with_totals && (query.group_by_with_rollup || query.group_by_with_cube))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING");
|
||||
|
||||
if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
|
||||
{
|
||||
query_info.projection->aggregate_overflow_row = aggregate_overflow_row;
|
||||
@ -1386,11 +1389,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
executeRollupOrCube(query_plan, Modificator::CUBE);
|
||||
|
||||
if ((query.group_by_with_rollup || query.group_by_with_cube || query.group_by_with_grouping_sets) && expressions.hasHaving())
|
||||
{
|
||||
if (query.group_by_with_totals)
|
||||
throw Exception("WITH TOTALS and WITH ROLLUP or CUBE or GROUPING SETS are not supported together in presence of HAVING", ErrorCodes::NOT_IMPLEMENTED);
|
||||
executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter);
|
||||
}
|
||||
}
|
||||
else if (expressions.hasHaving())
|
||||
executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter);
|
||||
|
@ -217,7 +217,7 @@ void TransactionLog::runUpdatingThread()
|
||||
try
|
||||
{
|
||||
/// Do not wait if we have some transactions to finalize
|
||||
if (!unknown_state_list_loaded.empty())
|
||||
if (unknown_state_list_loaded.empty())
|
||||
log_updated_event->wait();
|
||||
|
||||
if (stop_flag.load())
|
||||
|
@ -95,22 +95,22 @@ public:
|
||||
ASTPtr & refWhere() { return getExpression(Expression::WHERE); }
|
||||
ASTPtr & refHaving() { return getExpression(Expression::HAVING); }
|
||||
|
||||
const ASTPtr with() const { return getExpression(Expression::WITH); }
|
||||
const ASTPtr select() const { return getExpression(Expression::SELECT); }
|
||||
const ASTPtr tables() const { return getExpression(Expression::TABLES); }
|
||||
const ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); }
|
||||
const ASTPtr where() const { return getExpression(Expression::WHERE); }
|
||||
const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
||||
const ASTPtr having() const { return getExpression(Expression::HAVING); }
|
||||
const ASTPtr window() const { return getExpression(Expression::WINDOW); }
|
||||
const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
||||
const ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); }
|
||||
const ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); }
|
||||
const ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); }
|
||||
const ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); }
|
||||
const ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); }
|
||||
const ASTPtr settings() const { return getExpression(Expression::SETTINGS); }
|
||||
const ASTPtr interpolate() const { return getExpression(Expression::INTERPOLATE); }
|
||||
ASTPtr with() const { return getExpression(Expression::WITH); }
|
||||
ASTPtr select() const { return getExpression(Expression::SELECT); }
|
||||
ASTPtr tables() const { return getExpression(Expression::TABLES); }
|
||||
ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); }
|
||||
ASTPtr where() const { return getExpression(Expression::WHERE); }
|
||||
ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
||||
ASTPtr having() const { return getExpression(Expression::HAVING); }
|
||||
ASTPtr window() const { return getExpression(Expression::WINDOW); }
|
||||
ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
||||
ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); }
|
||||
ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); }
|
||||
ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); }
|
||||
ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); }
|
||||
ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); }
|
||||
ASTPtr settings() const { return getExpression(Expression::SETTINGS); }
|
||||
ASTPtr interpolate() const { return getExpression(Expression::INTERPOLATE); }
|
||||
|
||||
bool hasFiltration() const { return where() || prewhere() || having(); }
|
||||
|
||||
|
@ -803,6 +803,20 @@ namespace
|
||||
node = makeASTFunction("exists", subquery);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool parseGrouping(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr expr_list;
|
||||
if (!ParserExpressionList(false, false).parse(pos, expr_list, expected))
|
||||
return false;
|
||||
|
||||
auto res = std::make_shared<ASTFunction>();
|
||||
res->name = "grouping";
|
||||
res->arguments = expr_list;
|
||||
res->children.push_back(res->arguments);
|
||||
node = std::move(res);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -888,6 +902,8 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
else if (function_name_lowercase == "datediff" || function_name_lowercase == "date_diff"
|
||||
|| function_name_lowercase == "timestampdiff" || function_name_lowercase == "timestamp_diff")
|
||||
parsed_special_function = parseDateDiff(pos, node, expected);
|
||||
else if (function_name_lowercase == "grouping")
|
||||
parsed_special_function = parseGrouping(pos, node, expected);
|
||||
|
||||
if (parsed_special_function.has_value())
|
||||
return parsed_special_function.value() && ParserToken(TokenType::ClosingRoundBracket).ignore(pos);
|
||||
|
@ -12,7 +12,9 @@
|
||||
#include <Processors/Merges/FinishAggregatingInOrderTransform.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -33,6 +35,17 @@ static ITransformingStep::Traits getTraits()
|
||||
};
|
||||
}
|
||||
|
||||
Block appendGroupingSetColumn(Block header)
|
||||
{
|
||||
Block res;
|
||||
res.insert({std::make_shared<DataTypeUInt64>(), "__grouping_set"});
|
||||
|
||||
for (auto & col : header)
|
||||
res.insert(std::move(col));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
static Block appendGroupingColumn(Block block, const GroupingSetsParamsList & params)
|
||||
{
|
||||
if (params.empty())
|
||||
|
@ -25,6 +25,8 @@ struct GroupingSetsParams
|
||||
|
||||
using GroupingSetsParamsList = std::vector<GroupingSetsParams>;
|
||||
|
||||
Block appendGroupingSetColumn(Block header);
|
||||
|
||||
/// Aggregation. See AggregatingTransform.
|
||||
class AggregatingStep : public ITransformingStep
|
||||
{
|
||||
|
@ -1,6 +1,9 @@
|
||||
#include <Processors/QueryPlan/CubeStep.h>
|
||||
#include <Processors/Transforms/CubeTransform.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/QueryPlan/AggregatingStep.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -22,7 +25,8 @@ static ITransformingStep::Traits getTraits()
|
||||
}
|
||||
|
||||
CubeStep::CubeStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_)
|
||||
: ITransformingStep(input_stream_, params_->getHeader(), getTraits())
|
||||
: ITransformingStep(input_stream_, appendGroupingSetColumn(params_->getHeader()), getTraits())
|
||||
, keys_size(params_->params.keys_size)
|
||||
, params(std::move(params_))
|
||||
{
|
||||
/// Aggregation keys are distinct
|
||||
@ -30,14 +34,30 @@ CubeStep::CubeStep(const DataStream & input_stream_, AggregatingTransformParamsP
|
||||
output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name);
|
||||
}
|
||||
|
||||
void CubeStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
ProcessorPtr addGroupingSetForTotals(const Block & header, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number)
|
||||
{
|
||||
auto dag = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
|
||||
|
||||
auto grouping_col = ColumnUInt64::create(1, grouping_set_number);
|
||||
const auto * grouping_node = &dag->addColumn(
|
||||
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
|
||||
|
||||
grouping_node = &dag->materializeNode(*grouping_node);
|
||||
auto & index = dag->getIndex();
|
||||
index.insert(index.begin(), grouping_node);
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(dag, settings.getActionsSettings());
|
||||
return std::make_shared<ExpressionTransform>(header, expression);
|
||||
}
|
||||
|
||||
void CubeStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
pipeline.resize(1);
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
|
||||
{
|
||||
if (stream_type == QueryPipelineBuilder::StreamType::Totals)
|
||||
return nullptr;
|
||||
return addGroupingSetForTotals(header, settings, (UInt64(1) << keys_size) - 1);
|
||||
|
||||
return std::make_shared<CubeTransform>(header, std::move(params));
|
||||
});
|
||||
|
@ -21,6 +21,7 @@ public:
|
||||
|
||||
const Aggregator::Params & getParams() const;
|
||||
private:
|
||||
size_t keys_size;
|
||||
AggregatingTransformParamsPtr params;
|
||||
};
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/RollupStep.h>
|
||||
#include <Processors/Transforms/RollupTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/QueryPlan/AggregatingStep.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -22,22 +23,25 @@ static ITransformingStep::Traits getTraits()
|
||||
}
|
||||
|
||||
RollupStep::RollupStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_)
|
||||
: ITransformingStep(input_stream_, params_->getHeader(), getTraits())
|
||||
: ITransformingStep(input_stream_, appendGroupingSetColumn(params_->getHeader()), getTraits())
|
||||
, params(std::move(params_))
|
||||
, keys_size(params->params.keys_size)
|
||||
{
|
||||
/// Aggregation keys are distinct
|
||||
for (auto key : params->params.keys)
|
||||
output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name);
|
||||
}
|
||||
|
||||
void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
ProcessorPtr addGroupingSetForTotals(const Block & header, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number);
|
||||
|
||||
void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
pipeline.resize(1);
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
|
||||
{
|
||||
if (stream_type == QueryPipelineBuilder::StreamType::Totals)
|
||||
return nullptr;
|
||||
return addGroupingSetForTotals(header, settings, keys_size);
|
||||
|
||||
return std::make_shared<RollupTransform>(header, std::move(params));
|
||||
});
|
||||
|
@ -20,6 +20,7 @@ public:
|
||||
|
||||
private:
|
||||
AggregatingTransformParamsPtr params;
|
||||
size_t keys_size;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Processors/Transforms/CubeTransform.h>
|
||||
#include <Processors/Transforms/TotalsHavingTransform.h>
|
||||
#include <Processors/QueryPlan/AggregatingStep.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -9,7 +10,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
CubeTransform::CubeTransform(Block header, AggregatingTransformParamsPtr params_)
|
||||
: IAccumulatingTransform(std::move(header), params_->getHeader())
|
||||
: IAccumulatingTransform(std::move(header), appendGroupingSetColumn(params_->getHeader()))
|
||||
, params(std::move(params_))
|
||||
, keys(params->params.keys)
|
||||
, aggregates_mask(getAggregatesMask(params->getHeader(), params->params.aggregates))
|
||||
@ -75,6 +76,8 @@ Chunk CubeTransform::generate()
|
||||
}
|
||||
|
||||
finalizeChunk(gen_chunk, aggregates_mask);
|
||||
if (!gen_chunk.empty())
|
||||
gen_chunk.addColumn(0, ColumnUInt64::create(gen_chunk.getNumRows(), grouping_set++));
|
||||
return gen_chunk;
|
||||
}
|
||||
|
||||
|
@ -30,6 +30,7 @@ private:
|
||||
Columns current_zero_columns;
|
||||
|
||||
UInt64 mask = 0;
|
||||
UInt64 grouping_set = 0;
|
||||
|
||||
Chunk merge(Chunks && chunks, bool final);
|
||||
};
|
||||
|
@ -90,9 +90,9 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr &
|
||||
if (which.isDate() || which.isDate32())
|
||||
{
|
||||
Int64 avg_seconds = get<Int64>(descr.fill_step) * descr.step_kind->toAvgSeconds();
|
||||
if (avg_seconds < 86400)
|
||||
if (std::abs(avg_seconds) < 86400)
|
||||
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
|
||||
"Value of step is to low ({} seconds). Must be >= 1 day", avg_seconds);
|
||||
"Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds));
|
||||
}
|
||||
|
||||
if (which.isDate())
|
||||
|
@ -1,11 +1,12 @@
|
||||
#include <Processors/Transforms/RollupTransform.h>
|
||||
#include <Processors/Transforms/TotalsHavingTransform.h>
|
||||
#include <Processors/QueryPlan/AggregatingStep.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
RollupTransform::RollupTransform(Block header, AggregatingTransformParamsPtr params_)
|
||||
: IAccumulatingTransform(std::move(header), params_->getHeader())
|
||||
: IAccumulatingTransform(std::move(header), appendGroupingSetColumn(params_->getHeader()))
|
||||
, params(std::move(params_))
|
||||
, keys(params->params.keys)
|
||||
, aggregates_mask(getAggregatesMask(params->getHeader(), params->params.aggregates))
|
||||
@ -58,6 +59,8 @@ Chunk RollupTransform::generate()
|
||||
}
|
||||
|
||||
finalizeChunk(gen_chunk, aggregates_mask);
|
||||
if (!gen_chunk.empty())
|
||||
gen_chunk.addColumn(0, ColumnUInt64::create(gen_chunk.getNumRows(), set_counter++));
|
||||
return gen_chunk;
|
||||
}
|
||||
|
||||
|
@ -26,6 +26,7 @@ private:
|
||||
Chunks consumed_chunks;
|
||||
Chunk rollup_chunk;
|
||||
size_t last_removed_key = 0;
|
||||
size_t set_counter = 0;
|
||||
|
||||
Chunk merge(Chunks && chunks, bool final);
|
||||
};
|
||||
|
@ -146,6 +146,7 @@ StorageHDFS::StorageHDFS(
|
||||
, distributed_processing(distributed_processing_)
|
||||
, partition_by(partition_by_)
|
||||
{
|
||||
FormatFactory::instance().checkFormatName(format_name);
|
||||
context_->getRemoteHostFilter().checkURL(Poco::URI(uri_));
|
||||
checkHDFSURL(uri_);
|
||||
|
||||
|
@ -3079,7 +3079,9 @@ void MergeTreeData::forgetPartAndMoveToDetached(const MergeTreeData::DataPartPtr
|
||||
throw Exception("No such data part " + part_to_detach->getNameWithState(), ErrorCodes::NO_SUCH_DATA_PART);
|
||||
|
||||
/// What if part_to_detach is a reference to *it_part? Make a new owner just in case.
|
||||
const DataPartPtr & part = *it_part;
|
||||
/// Important to own part pointer here (not const reference), because it will be removed from data_parts_indexes
|
||||
/// few lines below.
|
||||
DataPartPtr part = *it_part; // NOLINT
|
||||
|
||||
if (part->getState() == DataPartState::Active)
|
||||
{
|
||||
|
@ -382,6 +382,8 @@ StorageFile::StorageFile(CommonArguments args)
|
||||
, compression_method(args.compression_method)
|
||||
, base_path(args.getContext()->getPath())
|
||||
{
|
||||
if (format_name != "Distributed")
|
||||
FormatFactory::instance().checkFormatName(format_name);
|
||||
}
|
||||
|
||||
void StorageFile::setStorageMetadata(CommonArguments args)
|
||||
|
@ -740,6 +740,7 @@ StorageS3::StorageS3(
|
||||
, partition_by(partition_by_)
|
||||
, is_key_with_globs(uri_.key.find_first_of("*?{") != std::string::npos)
|
||||
{
|
||||
FormatFactory::instance().checkFormatName(format_name);
|
||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri);
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
|
||||
|
@ -74,6 +74,7 @@ IStorageURLBase::IStorageURLBase(
|
||||
, http_method(http_method_)
|
||||
, partition_by(partition_by_)
|
||||
{
|
||||
FormatFactory::instance().checkFormatName(format_name);
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
|
||||
if (columns_.empty())
|
||||
|
@ -156,8 +156,13 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
|
||||
auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName());
|
||||
PreparedSets prepared_sets;
|
||||
SubqueriesForSets subqueries_for_sets;
|
||||
const NamesAndTypesList source_columns;
|
||||
const NamesAndTypesList aggregation_keys;
|
||||
const ColumnNumbersList grouping_set_keys;
|
||||
|
||||
ActionsVisitor::Data visitor_data(
|
||||
context, SizeLimits{}, 1, {}, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false);
|
||||
context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false,
|
||||
{ aggregation_keys, grouping_set_keys, GroupByKind::NONE });
|
||||
ActionsVisitor(visitor_data).visit(node);
|
||||
actions = visitor_data.getActions();
|
||||
auto expression_actions = std::make_shared<ExpressionActions>(actions);
|
||||
|
@ -33,7 +33,7 @@ def start_cluster():
|
||||
def test_config_with_hosts(start_cluster):
|
||||
assert (
|
||||
node1.query(
|
||||
"CREATE TABLE table_test_1_1 (word String) Engine=URL('http://host:80', HDFS)"
|
||||
"CREATE TABLE table_test_1_1 (word String) Engine=URL('http://host:80', CSV)"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
@ -44,7 +44,7 @@ def test_config_with_hosts(start_cluster):
|
||||
== ""
|
||||
)
|
||||
assert "not allowed" in node1.query_and_get_error(
|
||||
"CREATE TABLE table_test_1_4 (word String) Engine=URL('https://host:123', S3)"
|
||||
"CREATE TABLE table_test_1_4 (word String) Engine=URL('https://host:123', CSV)"
|
||||
)
|
||||
assert "not allowed" in node1.query_and_get_error(
|
||||
"CREATE TABLE table_test_1_4 (word String) Engine=URL('https://yandex2.ru', CSV)"
|
||||
@ -60,7 +60,7 @@ def test_config_with_only_primary_hosts(start_cluster):
|
||||
)
|
||||
assert (
|
||||
node2.query(
|
||||
"CREATE TABLE table_test_2_2 (word String) Engine=URL('https://host:123', S3)"
|
||||
"CREATE TABLE table_test_2_2 (word String) Engine=URL('https://host:123', CSV)"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
@ -72,25 +72,25 @@ def test_config_with_only_primary_hosts(start_cluster):
|
||||
)
|
||||
assert (
|
||||
node2.query(
|
||||
"CREATE TABLE table_test_2_4 (word String) Engine=URL('https://yandex.ru:87', HDFS)"
|
||||
"CREATE TABLE table_test_2_4 (word String) Engine=URL('https://yandex.ru:87', CSV)"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
assert "not allowed" in node2.query_and_get_error(
|
||||
"CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host', HDFS)"
|
||||
"CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host', CSV)"
|
||||
)
|
||||
assert "not allowed" in node2.query_and_get_error(
|
||||
"CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host:234', CSV)"
|
||||
)
|
||||
assert "not allowed" in node2.query_and_get_error(
|
||||
"CREATE TABLE table_test_2_6 (word String) Engine=URL('https://yandex2.ru', S3)"
|
||||
"CREATE TABLE table_test_2_6 (word String) Engine=URL('https://yandex2.ru', CSV)"
|
||||
)
|
||||
|
||||
|
||||
def test_config_with_only_regexp_hosts(start_cluster):
|
||||
assert (
|
||||
node3.query(
|
||||
"CREATE TABLE table_test_3_1 (word String) Engine=URL('https://host:80', HDFS)"
|
||||
"CREATE TABLE table_test_3_1 (word String) Engine=URL('https://host:80', CSV)"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
@ -104,7 +104,7 @@ def test_config_with_only_regexp_hosts(start_cluster):
|
||||
"CREATE TABLE table_test_3_3 (word String) Engine=URL('https://host', CSV)"
|
||||
)
|
||||
assert "not allowed" in node3.query_and_get_error(
|
||||
"CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)"
|
||||
"CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', CSV)"
|
||||
)
|
||||
|
||||
|
||||
@ -123,7 +123,7 @@ def test_config_without_allowed_hosts_section(start_cluster):
|
||||
)
|
||||
assert (
|
||||
node4.query(
|
||||
"CREATE TABLE table_test_4_3 (word String) Engine=URL('https://host', HDFS)"
|
||||
"CREATE TABLE table_test_4_3 (word String) Engine=URL('https://host', CSV)"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
@ -135,7 +135,7 @@ def test_config_without_allowed_hosts_section(start_cluster):
|
||||
)
|
||||
assert (
|
||||
node4.query(
|
||||
"CREATE TABLE table_test_4_5 (word String) Engine=URL('ftp://something.com', S3)"
|
||||
"CREATE TABLE table_test_4_5 (word String) Engine=URL('ftp://something.com', CSV)"
|
||||
)
|
||||
== ""
|
||||
)
|
||||
@ -149,13 +149,13 @@ def test_config_without_allowed_hosts(start_cluster):
|
||||
"CREATE TABLE table_test_5_2 (word String) Engine=S3('https://host:80/bucket/key', CSV)"
|
||||
)
|
||||
assert "not allowed" in node5.query_and_get_error(
|
||||
"CREATE TABLE table_test_5_3 (word String) Engine=URL('https://host', HDFS)"
|
||||
"CREATE TABLE table_test_5_3 (word String) Engine=URL('https://host', CSV)"
|
||||
)
|
||||
assert "not allowed" in node5.query_and_get_error(
|
||||
"CREATE TABLE table_test_5_4 (word String) Engine=URL('https://yandex.ru', CSV)"
|
||||
)
|
||||
assert "not allowed" in node5.query_and_get_error(
|
||||
"CREATE TABLE table_test_5_5 (word String) Engine=URL('ftp://something.com', S3)"
|
||||
"CREATE TABLE table_test_5_5 (word String) Engine=URL('ftp://something.com', CSV)"
|
||||
)
|
||||
|
||||
|
||||
|
@ -107,3 +107,6 @@
|
||||
2020-05-01 2 0
|
||||
2020-05-01 3 0
|
||||
2020-05-01 4 0
|
||||
1970-01-04
|
||||
1970-01-03
|
||||
1970-01-02
|
||||
|
@ -79,3 +79,6 @@ d WITH FILL
|
||||
id WITH FILL FROM 1 TO 5;
|
||||
|
||||
DROP TABLE with_fill_date;
|
||||
|
||||
SELECT d FROM (SELECT toDate(1) AS d)
|
||||
ORDER BY d DESC WITH FILL FROM toDate(3) TO toDate(0) STEP INTERVAL -1 DAY;
|
||||
|
176
tests/queries/0_stateless/02293_grouping_function.reference
Normal file
176
tests/queries/0_stateless/02293_grouping_function.reference
Normal file
@ -0,0 +1,176 @@
|
||||
-- { echoOn }
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr;
|
||||
0 1
|
||||
0 1
|
||||
0 2
|
||||
1 2
|
||||
2 2
|
||||
3 2
|
||||
4 2
|
||||
5 2
|
||||
6 2
|
||||
7 2
|
||||
8 2
|
||||
9 2
|
||||
SELECT
|
||||
number,
|
||||
grouping(number % 2, number) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr;
|
||||
0 1
|
||||
0 2
|
||||
0 2
|
||||
1 1
|
||||
2 1
|
||||
3 1
|
||||
4 1
|
||||
5 1
|
||||
6 1
|
||||
7 1
|
||||
8 1
|
||||
9 1
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) = 1 AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr;
|
||||
0 0
|
||||
0 1
|
||||
0 1
|
||||
1 0
|
||||
2 0
|
||||
3 0
|
||||
4 0
|
||||
5 0
|
||||
6 0
|
||||
7 0
|
||||
8 0
|
||||
9 0
|
||||
SELECT
|
||||
number
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, grouping(number, number % 2) = 1;
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
||||
SELECT
|
||||
number,
|
||||
count(),
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number, number % 2),
|
||||
()
|
||||
)
|
||||
ORDER BY (gr, number);
|
||||
0 10 0
|
||||
0 1 2
|
||||
1 1 2
|
||||
2 1 2
|
||||
3 1 2
|
||||
4 1 2
|
||||
5 1 2
|
||||
6 1 2
|
||||
7 1 2
|
||||
8 1 2
|
||||
9 1 2
|
||||
0 1 3
|
||||
1 1 3
|
||||
2 1 3
|
||||
3 1 3
|
||||
4 1 3
|
||||
5 1 3
|
||||
6 1 3
|
||||
7 1 3
|
||||
8 1 3
|
||||
9 1 3
|
||||
SELECT
|
||||
number
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
HAVING grouping(number, number % 2) = 2
|
||||
ORDER BY number
|
||||
SETTINGS enable_optimize_predicate_expression = 0;
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
||||
SELECT
|
||||
number
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
HAVING grouping(number, number % 2) = 1
|
||||
ORDER BY number
|
||||
SETTINGS enable_optimize_predicate_expression = 0;
|
||||
0
|
||||
0
|
||||
SELECT
|
||||
number,
|
||||
GROUPING(number, number % 2) = 1 as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2))
|
||||
ORDER BY number, gr;
|
||||
0 0
|
||||
0 1
|
||||
0 1
|
||||
1 0
|
||||
2 0
|
||||
3 0
|
||||
4 0
|
||||
5 0
|
||||
6 0
|
||||
7 0
|
||||
8 0
|
||||
9 0
|
101
tests/queries/0_stateless/02293_grouping_function.sql
Normal file
101
tests/queries/0_stateless/02293_grouping_function.sql
Normal file
@ -0,0 +1,101 @@
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2, number % 3) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr; -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
-- { echoOn }
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number % 2, number) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) = 1 AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, gr;
|
||||
|
||||
SELECT
|
||||
number
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
ORDER BY number, grouping(number, number % 2) = 1;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
count(),
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number, number % 2),
|
||||
()
|
||||
)
|
||||
ORDER BY (gr, number);
|
||||
|
||||
SELECT
|
||||
number
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
HAVING grouping(number, number % 2) = 2
|
||||
ORDER BY number
|
||||
SETTINGS enable_optimize_predicate_expression = 0;
|
||||
|
||||
SELECT
|
||||
number
|
||||
FROM numbers(10)
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2)
|
||||
)
|
||||
HAVING grouping(number, number % 2) = 1
|
||||
ORDER BY number
|
||||
SETTINGS enable_optimize_predicate_expression = 0;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
GROUPING(number, number % 2) = 1 as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(number),
|
||||
(number % 2))
|
||||
ORDER BY number, gr;
|
@ -0,0 +1,273 @@
|
||||
-- { echoOn }
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) = 3
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
ORDER BY number;
|
||||
0 1
|
||||
1 1
|
||||
2 1
|
||||
3 1
|
||||
4 1
|
||||
5 1
|
||||
6 1
|
||||
7 1
|
||||
8 1
|
||||
9 1
|
||||
SELECT
|
||||
number,
|
||||
grouping(number),
|
||||
GROUPING(number % 2)
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
ORDER BY number;
|
||||
0 1 1
|
||||
1 1 1
|
||||
2 1 1
|
||||
3 1 1
|
||||
4 1 1
|
||||
5 1 1
|
||||
6 1 1
|
||||
7 1 1
|
||||
8 1 1
|
||||
9 1 1
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
WITH ROLLUP
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 0
|
||||
0 2
|
||||
0 3
|
||||
1 2
|
||||
1 3
|
||||
2 2
|
||||
2 3
|
||||
3 2
|
||||
3 3
|
||||
4 2
|
||||
4 3
|
||||
5 2
|
||||
5 3
|
||||
6 2
|
||||
6 3
|
||||
7 2
|
||||
7 3
|
||||
8 2
|
||||
8 3
|
||||
9 2
|
||||
9 3
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
ROLLUP(number, number % 2)
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 0
|
||||
0 2
|
||||
0 3
|
||||
1 2
|
||||
1 3
|
||||
2 2
|
||||
2 3
|
||||
3 2
|
||||
3 3
|
||||
4 2
|
||||
4 3
|
||||
5 2
|
||||
5 3
|
||||
6 2
|
||||
6 3
|
||||
7 2
|
||||
7 3
|
||||
8 2
|
||||
8 3
|
||||
9 2
|
||||
9 3
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
WITH CUBE
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 0
|
||||
0 1
|
||||
0 1
|
||||
0 2
|
||||
0 3
|
||||
1 2
|
||||
1 3
|
||||
2 2
|
||||
2 3
|
||||
3 2
|
||||
3 3
|
||||
4 2
|
||||
4 3
|
||||
5 2
|
||||
5 3
|
||||
6 2
|
||||
6 3
|
||||
7 2
|
||||
7 3
|
||||
8 2
|
||||
8 3
|
||||
9 2
|
||||
9 3
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2)
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 0
|
||||
0 1
|
||||
0 1
|
||||
0 2
|
||||
0 3
|
||||
1 2
|
||||
1 3
|
||||
2 2
|
||||
2 3
|
||||
3 2
|
||||
3 3
|
||||
4 2
|
||||
4 3
|
||||
5 2
|
||||
5 3
|
||||
6 2
|
||||
6 3
|
||||
7 2
|
||||
7 3
|
||||
8 2
|
||||
8 3
|
||||
9 2
|
||||
9 3
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) + 3 as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2)
|
||||
HAVING grouping(number) != 0
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 5
|
||||
0 6
|
||||
1 5
|
||||
1 6
|
||||
2 5
|
||||
2 6
|
||||
3 5
|
||||
3 6
|
||||
4 5
|
||||
4 6
|
||||
5 5
|
||||
5 6
|
||||
6 5
|
||||
6 6
|
||||
7 5
|
||||
7 6
|
||||
8 5
|
||||
8 6
|
||||
9 5
|
||||
9 6
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2) WITH TOTALS
|
||||
HAVING grouping(number) != 0
|
||||
ORDER BY
|
||||
number, gr; -- { serverError NOT_IMPLEMENTED }
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2) WITH TOTALS
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 0
|
||||
0 1
|
||||
0 1
|
||||
0 2
|
||||
0 3
|
||||
1 2
|
||||
1 3
|
||||
2 2
|
||||
2 3
|
||||
3 2
|
||||
3 3
|
||||
4 2
|
||||
4 3
|
||||
5 2
|
||||
5 3
|
||||
6 2
|
||||
6 3
|
||||
7 2
|
||||
7 3
|
||||
8 2
|
||||
8 3
|
||||
9 2
|
||||
9 3
|
||||
|
||||
0 0
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
ROLLUP(number, number % 2) WITH TOTALS
|
||||
HAVING grouping(number) != 0
|
||||
ORDER BY
|
||||
number, gr; -- { serverError NOT_IMPLEMENTED }
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
ROLLUP(number, number % 2) WITH TOTALS
|
||||
ORDER BY
|
||||
number, gr;
|
||||
0 0
|
||||
0 2
|
||||
0 3
|
||||
1 2
|
||||
1 3
|
||||
2 2
|
||||
2 3
|
||||
3 2
|
||||
3 3
|
||||
4 2
|
||||
4 3
|
||||
5 2
|
||||
5 3
|
||||
6 2
|
||||
6 3
|
||||
7 2
|
||||
7 3
|
||||
8 2
|
||||
8 3
|
||||
9 2
|
||||
9 3
|
||||
|
||||
0 0
|
116
tests/queries/0_stateless/02293_grouping_function_group_by.sql
Normal file
116
tests/queries/0_stateless/02293_grouping_function_group_by.sql
Normal file
@ -0,0 +1,116 @@
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2, number % 3) = 6
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
ORDER BY number; -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
-- { echoOn }
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) = 3
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
ORDER BY number;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number),
|
||||
GROUPING(number % 2)
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
ORDER BY number;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
WITH ROLLUP
|
||||
ORDER BY
|
||||
number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
ROLLUP(number, number % 2)
|
||||
ORDER BY
|
||||
number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
number,
|
||||
number % 2
|
||||
WITH CUBE
|
||||
ORDER BY
|
||||
number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) AS gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2)
|
||||
ORDER BY
|
||||
number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) + 3 as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2)
|
||||
HAVING grouping(number) != 0
|
||||
ORDER BY
|
||||
number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2) WITH TOTALS
|
||||
HAVING grouping(number) != 0
|
||||
ORDER BY
|
||||
number, gr; -- { serverError NOT_IMPLEMENTED }
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
CUBE(number, number % 2) WITH TOTALS
|
||||
ORDER BY
|
||||
number, gr;
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
ROLLUP(number, number % 2) WITH TOTALS
|
||||
HAVING grouping(number) != 0
|
||||
ORDER BY
|
||||
number, gr; -- { serverError NOT_IMPLEMENTED }
|
||||
|
||||
SELECT
|
||||
number,
|
||||
grouping(number, number % 2) as gr
|
||||
FROM remote('127.0.0.{2,3}', numbers(10))
|
||||
GROUP BY
|
||||
ROLLUP(number, number % 2) WITH TOTALS
|
||||
ORDER BY
|
||||
number, gr;
|
@ -0,0 +1,6 @@
|
||||
-- Tags: no-fasttest, use-hdfs, no-backward-compatibility-check:22.5
|
||||
|
||||
create table test_02311 (x UInt32) engine=File(UnknownFormat); -- {serverError UNKNOWN_FORMAT}
|
||||
create table test_02311 (x UInt32) engine=URL('http://some/url', UnknownFormat); -- {serverError UNKNOWN_FORMAT}
|
||||
create table test_02311 (x UInt32) engine=S3('http://host:2020/test/data', UnknownFormat); -- {serverError UNKNOWN_FORMAT}
|
||||
create table test_02311 (x UInt32) engine=HDFS('http://hdfs:9000/data', UnknownFormat); -- {serverError UNKNOWN_FORMAT}
|
Loading…
Reference in New Issue
Block a user