mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into CLICKHOUSE-4112
This commit is contained in:
commit
57ea89c5f3
@ -2,8 +2,6 @@
|
||||
|
||||
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
|
||||
|
||||
🎤🥂 **ClickHouse Meetup in [Amsterdam on November 15](https://events.yandex.com/events/meetings/15-11-2018/)** 🍰🔥🐻
|
||||
|
||||
## Useful Links
|
||||
|
||||
* [Official website](https://clickhouse.yandex/) has quick high-level overview of ClickHouse on main page.
|
||||
|
2
contrib/ssl
vendored
2
contrib/ssl
vendored
@ -1 +1 @@
|
||||
Subproject commit 919f6f1331d500bfdd26f8bbbf88e92c0119879b
|
||||
Subproject commit dbbbcdbbd17785566f8f9c107b714f9e213d7293
|
@ -86,7 +86,6 @@ list (APPEND dbms_sources
|
||||
src/AggregateFunctions/AggregateFunctionFactory.cpp
|
||||
src/AggregateFunctions/AggregateFunctionCombinatorFactory.cpp
|
||||
src/AggregateFunctions/AggregateFunctionState.cpp
|
||||
src/AggregateFunctions/FactoryHelpers.cpp
|
||||
src/AggregateFunctions/parseAggregateFunctionParameters.cpp)
|
||||
|
||||
list (APPEND dbms_headers
|
||||
|
@ -1,11 +1,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionQuantile.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -52,13 +48,11 @@ static constexpr bool SupportDecimal()
|
||||
}
|
||||
|
||||
|
||||
template <template <typename> class Function, bool have_second_arg>
|
||||
template <template <typename> class Function>
|
||||
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types, const Array & params)
|
||||
{
|
||||
if constexpr (have_second_arg)
|
||||
assertBinary(name, argument_types);
|
||||
else
|
||||
assertUnary(name, argument_types);
|
||||
/// Second argument type check doesn't depend on the type of the first one.
|
||||
Function<void>::assertSecondArg(argument_types);
|
||||
|
||||
const DataTypePtr & argument_type = argument_types[0];
|
||||
WhichDataType which(argument_type);
|
||||
@ -86,29 +80,29 @@ AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, c
|
||||
|
||||
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction(NameQuantile::name, createAggregateFunctionQuantile<FuncQuantile, false>);
|
||||
factory.registerFunction(NameQuantiles::name, createAggregateFunctionQuantile<FuncQuantiles, false>);
|
||||
factory.registerFunction(NameQuantile::name, createAggregateFunctionQuantile<FuncQuantile>);
|
||||
factory.registerFunction(NameQuantiles::name, createAggregateFunctionQuantile<FuncQuantiles>);
|
||||
|
||||
factory.registerFunction(NameQuantileDeterministic::name, createAggregateFunctionQuantile<FuncQuantileDeterministic, true>);
|
||||
factory.registerFunction(NameQuantilesDeterministic::name, createAggregateFunctionQuantile<FuncQuantilesDeterministic, true>);
|
||||
factory.registerFunction(NameQuantileDeterministic::name, createAggregateFunctionQuantile<FuncQuantileDeterministic>);
|
||||
factory.registerFunction(NameQuantilesDeterministic::name, createAggregateFunctionQuantile<FuncQuantilesDeterministic>);
|
||||
|
||||
factory.registerFunction(NameQuantileExact::name, createAggregateFunctionQuantile<FuncQuantileExact, false>);
|
||||
factory.registerFunction(NameQuantilesExact::name, createAggregateFunctionQuantile<FuncQuantilesExact, false>);
|
||||
factory.registerFunction(NameQuantileExact::name, createAggregateFunctionQuantile<FuncQuantileExact>);
|
||||
factory.registerFunction(NameQuantilesExact::name, createAggregateFunctionQuantile<FuncQuantilesExact>);
|
||||
|
||||
factory.registerFunction(NameQuantileExactWeighted::name, createAggregateFunctionQuantile<FuncQuantileExactWeighted, true>);
|
||||
factory.registerFunction(NameQuantilesExactWeighted::name, createAggregateFunctionQuantile<FuncQuantilesExactWeighted, true>);
|
||||
factory.registerFunction(NameQuantileExactWeighted::name, createAggregateFunctionQuantile<FuncQuantileExactWeighted>);
|
||||
factory.registerFunction(NameQuantilesExactWeighted::name, createAggregateFunctionQuantile<FuncQuantilesExactWeighted>);
|
||||
|
||||
factory.registerFunction(NameQuantileTiming::name, createAggregateFunctionQuantile<FuncQuantileTiming, false>);
|
||||
factory.registerFunction(NameQuantilesTiming::name, createAggregateFunctionQuantile<FuncQuantilesTiming, false>);
|
||||
factory.registerFunction(NameQuantileTiming::name, createAggregateFunctionQuantile<FuncQuantileTiming>);
|
||||
factory.registerFunction(NameQuantilesTiming::name, createAggregateFunctionQuantile<FuncQuantilesTiming>);
|
||||
|
||||
factory.registerFunction(NameQuantileTimingWeighted::name, createAggregateFunctionQuantile<FuncQuantileTimingWeighted, true>);
|
||||
factory.registerFunction(NameQuantilesTimingWeighted::name, createAggregateFunctionQuantile<FuncQuantilesTimingWeighted, true>);
|
||||
factory.registerFunction(NameQuantileTimingWeighted::name, createAggregateFunctionQuantile<FuncQuantileTimingWeighted>);
|
||||
factory.registerFunction(NameQuantilesTimingWeighted::name, createAggregateFunctionQuantile<FuncQuantilesTimingWeighted>);
|
||||
|
||||
factory.registerFunction(NameQuantileTDigest::name, createAggregateFunctionQuantile<FuncQuantileTDigest, false>);
|
||||
factory.registerFunction(NameQuantilesTDigest::name, createAggregateFunctionQuantile<FuncQuantilesTDigest, false>);
|
||||
factory.registerFunction(NameQuantileTDigest::name, createAggregateFunctionQuantile<FuncQuantileTDigest>);
|
||||
factory.registerFunction(NameQuantilesTDigest::name, createAggregateFunctionQuantile<FuncQuantilesTDigest>);
|
||||
|
||||
factory.registerFunction(NameQuantileTDigestWeighted::name, createAggregateFunctionQuantile<FuncQuantileTDigestWeighted, true>);
|
||||
factory.registerFunction(NameQuantilesTDigestWeighted::name, createAggregateFunctionQuantile<FuncQuantilesTDigestWeighted, true>);
|
||||
factory.registerFunction(NameQuantileTDigestWeighted::name, createAggregateFunctionQuantile<FuncQuantileTDigestWeighted>);
|
||||
factory.registerFunction(NameQuantilesTDigestWeighted::name, createAggregateFunctionQuantile<FuncQuantilesTDigestWeighted>);
|
||||
|
||||
/// 'median' is an alias for 'quantile'
|
||||
factory.registerAlias("median", NameQuantile::name);
|
||||
|
@ -1,19 +1,28 @@
|
||||
#pragma once
|
||||
|
||||
#include <type_traits>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
/// These must be exposed in header for the purpose of dynamic compilation.
|
||||
#include <AggregateFunctions/QuantileReservoirSampler.h>
|
||||
#include <AggregateFunctions/QuantileReservoirSamplerDeterministic.h>
|
||||
#include <AggregateFunctions/QuantileExact.h>
|
||||
#include <AggregateFunctions/QuantileExactWeighted.h>
|
||||
#include <AggregateFunctions/QuantileTiming.h>
|
||||
#include <AggregateFunctions/QuantileTDigest.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/QuantilesCommon.h>
|
||||
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -36,10 +45,10 @@ template <
|
||||
typename Data,
|
||||
/// Structure with static member "name", containing the name of the aggregate function.
|
||||
typename Name,
|
||||
/// If true, the function accept second argument
|
||||
/// If true, the function accepts the second argument
|
||||
/// (in can be "weight" to calculate quantiles or "determinator" that is used instead of PRNG).
|
||||
/// Second argument is always obtained through 'getUInt' method.
|
||||
bool have_second_arg,
|
||||
bool has_second_arg,
|
||||
/// If non-void, the function will return float of specified type with possibly interpolated results and NaN if there was no values.
|
||||
/// Otherwise it will return Value type and default value if there was no values.
|
||||
/// As an example, the function cannot return floats, if the SQL type of argument is Date or DateTime.
|
||||
@ -49,12 +58,14 @@ template <
|
||||
bool returns_many
|
||||
>
|
||||
class AggregateFunctionQuantile final : public IAggregateFunctionDataHelper<Data,
|
||||
AggregateFunctionQuantile<Value, Data, Name, have_second_arg, FloatReturnType, returns_many>>
|
||||
AggregateFunctionQuantile<Value, Data, Name, has_second_arg, FloatReturnType, returns_many>>
|
||||
{
|
||||
private:
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<Value>, ColumnDecimal<Value>, ColumnVector<Value>>;
|
||||
|
||||
static constexpr bool returns_float = !std::is_same_v<FloatReturnType, void>;
|
||||
static constexpr bool returns_float = !(std::is_same_v<FloatReturnType, void>)
|
||||
&& (!(std::is_same_v<Value, DataTypeDate::FieldType> || std::is_same_v<Value, DataTypeDateTime::FieldType>)
|
||||
|| std::is_same_v<Data, QuantileTiming<Value>>);
|
||||
static_assert(!IsDecimalNumber<Value> || !returns_float);
|
||||
|
||||
QuantileLevels<Float64> levels;
|
||||
@ -92,7 +103,7 @@ public:
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
|
||||
{
|
||||
const auto & column = static_cast<const ColVecType &>(*columns[0]);
|
||||
if constexpr (have_second_arg)
|
||||
if constexpr (has_second_arg)
|
||||
this->data(place).add(
|
||||
column.getData()[row_num],
|
||||
columns[1]->getUInt(row_num));
|
||||
@ -159,22 +170,17 @@ public:
|
||||
}
|
||||
|
||||
const char * getHeaderFilePath() const override { return __FILE__; }
|
||||
|
||||
static void assertSecondArg(const DataTypes & argument_types)
|
||||
{
|
||||
if constexpr (has_second_arg)
|
||||
/// TODO: check that second argument is of numerical type.
|
||||
assertBinary(Name::name, argument_types);
|
||||
else
|
||||
assertUnary(Name::name, argument_types);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
/// These must be exposed in header for the purpose of dynamic compilation.
|
||||
#include <AggregateFunctions/QuantileReservoirSampler.h>
|
||||
#include <AggregateFunctions/QuantileReservoirSamplerDeterministic.h>
|
||||
#include <AggregateFunctions/QuantileExact.h>
|
||||
#include <AggregateFunctions/QuantileExactWeighted.h>
|
||||
#include <AggregateFunctions/QuantileTiming.h>
|
||||
#include <AggregateFunctions/QuantileTDigest.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameQuantile { static constexpr auto name = "quantile"; };
|
||||
struct NameQuantiles { static constexpr auto name = "quantiles"; };
|
||||
struct NameQuantileDeterministic { static constexpr auto name = "quantileDeterministic"; };
|
||||
|
@ -1,31 +0,0 @@
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
void assertNoParameters(const std::string & name, const Array & parameters)
|
||||
{
|
||||
if (!parameters.empty())
|
||||
throw Exception("Aggregate function " + name + " cannot have parameters", ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS);
|
||||
}
|
||||
|
||||
void assertUnary(const std::string & name, const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Aggregate function " + name + " require single argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
void assertBinary(const std::string & name, const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Aggregate function " + name + " require two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
}
|
@ -7,8 +7,28 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void assertNoParameters(const std::string & name, const Array & parameters);
|
||||
void assertUnary(const std::string & name, const DataTypes & argument_types);
|
||||
void assertBinary(const std::string & name, const DataTypes & argument_types);
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
inline void assertNoParameters(const std::string & name, const Array & parameters)
|
||||
{
|
||||
if (!parameters.empty())
|
||||
throw Exception("Aggregate function " + name + " cannot have parameters", ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS);
|
||||
}
|
||||
|
||||
inline void assertUnary(const std::string & name, const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Aggregate function " + name + " require single argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
inline void assertBinary(const std::string & name, const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Aggregate function " + name + " require two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <vector>
|
||||
#include <string>
|
||||
#include <set>
|
||||
#include <unordered_set>
|
||||
#include <unordered_map>
|
||||
|
||||
@ -11,6 +12,7 @@ namespace DB
|
||||
|
||||
using Names = std::vector<std::string>;
|
||||
using NameSet = std::unordered_set<std::string>;
|
||||
using NameOrderedSet = std::set<std::string>;
|
||||
using NameToNameMap = std::unordered_map<std::string, std::string>;
|
||||
using NameToNameSetMap = std::unordered_map<std::string, NameSet>;
|
||||
|
||||
|
@ -177,7 +177,7 @@ void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr,
|
||||
ColumnString::Chars_t & data = column_string.getChars();
|
||||
ColumnString::Offsets & offsets = column_string.getOffsets();
|
||||
|
||||
double avg_chars_size;
|
||||
double avg_chars_size = 1; /// By default reserve only for empty strings.
|
||||
|
||||
if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0]))
|
||||
{
|
||||
@ -186,13 +186,19 @@ void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr,
|
||||
|
||||
avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// By default reserve only for empty strings.
|
||||
avg_chars_size = 1;
|
||||
}
|
||||
|
||||
data.reserve(data.size() + std::ceil(limit * avg_chars_size));
|
||||
try
|
||||
{
|
||||
data.reserve(data.size() + std::ceil(limit * avg_chars_size));
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(
|
||||
"avg_value_size_hint = " + toString(avg_value_size_hint)
|
||||
+ ", avg_chars_size = " + toString(avg_chars_size)
|
||||
+ ", limit = " + toString(limit));
|
||||
throw;
|
||||
}
|
||||
|
||||
offsets.reserve(offsets.size() + limit);
|
||||
|
||||
|
@ -301,7 +301,11 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason)
|
||||
for (const HostID & host : task->entry.hosts)
|
||||
{
|
||||
auto maybe_secure_port = context.getTCPPortSecure();
|
||||
bool is_local_port = maybe_secure_port ? host.isLocalAddress(*maybe_secure_port) : host.isLocalAddress(context.getTCPPort());
|
||||
|
||||
/// The port is considered local if it matches TCP or TCP secure port that the server is listening.
|
||||
bool is_local_port = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port))
|
||||
|| host.isLocalAddress(context.getTCPPort());
|
||||
|
||||
if (!is_local_port)
|
||||
continue;
|
||||
|
||||
|
@ -350,7 +350,6 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
||||
return columns_list;
|
||||
}
|
||||
|
||||
|
||||
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpressionList & columns, const Context & context)
|
||||
{
|
||||
ColumnsDescription res;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Names.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/RWLockFIFO.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Storages/ITableDeclaration.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
@ -349,6 +350,24 @@ public:
|
||||
/// Returns primary expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getPrimaryExpression() const { return nullptr; }
|
||||
|
||||
/// Returns partition expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getPartitionExpression() const { return nullptr; }
|
||||
|
||||
/// Returns secondary expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getOrderExpression() const { return nullptr; }
|
||||
|
||||
/// Returns sampling key names for storage or empty vector if there is no.
|
||||
virtual Names getSamplingExpressionNames() const { return {}; }
|
||||
|
||||
/// Returns primary key names for storage or empty vector if there is no.
|
||||
virtual Names getPrimaryExpressionNames() const { return {}; }
|
||||
|
||||
/// Returns partition key names for storage or empty vector if there is no.
|
||||
virtual Names getPartitionExpressionNames() const { return {}; }
|
||||
|
||||
/// Returns order key names for storage or empty vector if there is no.
|
||||
virtual Names getOrderExpressionNames() const { return {}; }
|
||||
|
||||
using ITableDeclaration::ITableDeclaration;
|
||||
using std::enable_shared_from_this<IStorage>::shared_from_this;
|
||||
|
||||
|
@ -938,4 +938,34 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
|
||||
return {};
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getSamplingExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.sampling_expression;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.begin(), names.end());
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getPrimaryExpressionNames() const
|
||||
{
|
||||
return data.getPrimarySortColumns();
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getPartitionExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.partition_expr_ast;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.cbegin(), names.cend());
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getOrderExpressionNames() const
|
||||
{
|
||||
return data.getSortColumns();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
@ -98,6 +99,17 @@ public:
|
||||
|
||||
ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; }
|
||||
|
||||
ASTPtr getPartitionExpression() const override { return data.partition_expr_ast; }
|
||||
|
||||
ASTPtr getOrderExpression() const override { return data.secondary_sort_expr_ast; }
|
||||
|
||||
Names getSamplingExpressionNames() const override;
|
||||
|
||||
Names getPrimaryExpressionNames() const override;
|
||||
|
||||
Names getPartitionExpressionNames() const override;
|
||||
|
||||
Names getOrderExpressionNames() const override;
|
||||
private:
|
||||
String path;
|
||||
String database_name;
|
||||
|
@ -4856,4 +4856,34 @@ bool StorageReplicatedMergeTree::dropPartsInPartition(
|
||||
return true;
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getSamplingExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.sampling_expression;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.begin(), names.end());
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getPrimaryExpressionNames() const
|
||||
{
|
||||
return data.getPrimarySortColumns();
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getOrderExpressionNames() const
|
||||
{
|
||||
return data.getSortColumns();
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getPartitionExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.partition_expr_ast;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.cbegin(), names.cend());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -197,6 +197,14 @@ public:
|
||||
|
||||
ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; }
|
||||
|
||||
Names getSamplingExpressionNames() const override;
|
||||
|
||||
Names getPrimaryExpressionNames() const override;
|
||||
|
||||
Names getOrderExpressionNames() const override;
|
||||
|
||||
Names getPartitionExpressionNames() const override;
|
||||
|
||||
private:
|
||||
/// Delete old parts from disk and from ZooKeeper.
|
||||
void clearOldPartsAndRemoveFromZK();
|
||||
|
@ -36,6 +36,10 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
|
||||
{ "data_compressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "data_uncompressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "marks_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_order_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_partition_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_sample_key", std::make_shared<DataTypeUInt8>() },
|
||||
}));
|
||||
}
|
||||
|
||||
@ -80,6 +84,10 @@ protected:
|
||||
|
||||
NamesAndTypesList columns;
|
||||
ColumnDefaults column_defaults;
|
||||
Names partition_key_names;
|
||||
Names order_key_names;
|
||||
Names primary_key_names;
|
||||
Names sampling_key_names;
|
||||
MergeTreeData::ColumnSizeByName column_sizes;
|
||||
|
||||
{
|
||||
@ -106,6 +114,11 @@ protected:
|
||||
columns = storage->getColumns().getAll();
|
||||
column_defaults = storage->getColumns().defaults;
|
||||
|
||||
partition_key_names = storage->getPartitionExpressionNames();
|
||||
order_key_names = storage->getOrderExpressionNames();
|
||||
primary_key_names = storage->getPrimaryExpressionNames();
|
||||
sampling_key_names = storage->getSamplingExpressionNames();
|
||||
|
||||
/** Info about sizes of columns for tables of MergeTree family.
|
||||
* NOTE: It is possible to add getter for this info to IStorage interface.
|
||||
*/
|
||||
@ -173,6 +186,22 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
auto find_in_vector = [&key = column.name](const Names& names)
|
||||
{
|
||||
return std::find(names.cbegin(), names.cend(), key) != names.end();
|
||||
};
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(primary_key_names));
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(order_key_names));
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(partition_key_names));
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(sampling_key_names));
|
||||
}
|
||||
|
||||
++rows_count;
|
||||
}
|
||||
}
|
||||
|
@ -39,7 +39,11 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
|
||||
{"dependencies_database", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"dependencies_table", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"create_table_query", std::make_shared<DataTypeString>()},
|
||||
{"engine_full", std::make_shared<DataTypeString>()}
|
||||
{"engine_full", std::make_shared<DataTypeString>()},
|
||||
{"primary_key", std::make_shared<DataTypeString>()},
|
||||
{"order_key", std::make_shared<DataTypeString>()},
|
||||
{"partition_key", std::make_shared<DataTypeString>()},
|
||||
{"sample_key", std::make_shared<DataTypeString>()},
|
||||
}));
|
||||
}
|
||||
|
||||
@ -65,7 +69,7 @@ public:
|
||||
size_t max_block_size,
|
||||
ColumnPtr databases,
|
||||
const Context & context)
|
||||
: columns_mask(columns_mask), header(header), max_block_size(max_block_size), databases(std::move(databases)), context(context) {}
|
||||
: columns_mask(std::move(columns_mask)), header(std::move(header)), max_block_size(max_block_size), databases(std::move(databases)), context(context) {}
|
||||
|
||||
String getName() const override { return "Tables"; }
|
||||
Block getHeader() const override { return header; }
|
||||
@ -144,6 +148,18 @@ protected:
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(table.second->getName());
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
}
|
||||
|
||||
@ -173,7 +189,7 @@ protected:
|
||||
res_columns[res_index++]->insert(tables_it->table()->getName());
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(0u);
|
||||
res_columns[res_index++]->insert(0u); // is_temporary
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(tables_it->table()->getDataPath());
|
||||
@ -234,6 +250,42 @@ protected:
|
||||
res_columns[res_index++]->insert(engine_full);
|
||||
}
|
||||
}
|
||||
else
|
||||
src_index += 2;
|
||||
|
||||
const auto table_it = context.getTable(database_name, table_name);
|
||||
ASTPtr expression_ptr;
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getPrimaryExpression()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getOrderExpression()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getPartitionExpression()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getSamplingExpression()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
t Memory 1 0000-00-00 00:00:00 [] [] Memory
|
||||
t Memory 1 0000-00-00 00:00:00 [] [] Memory
|
||||
1
|
||||
1
|
||||
1
|
||||
|
@ -0,0 +1,16 @@
|
||||
2016-06-15 23:00:00
|
||||
['2016-06-15 23:00:00']
|
||||
2016-06-15 23:00:00
|
||||
['2016-06-15 23:00:00']
|
||||
2016-06-15 23:00:00
|
||||
['2016-06-15 23:00:00']
|
||||
2016-06-15 23:00:00
|
||||
['2016-06-15 23:00:00']
|
||||
30000
|
||||
[30000]
|
||||
30000
|
||||
[30000]
|
||||
2016-06-15 23:01:04
|
||||
['2016-06-15 23:01:04']
|
||||
2016-06-15 23:01:04
|
||||
['2016-06-15 23:01:04']
|
26
dbms/tests/queries/0_stateless/00753_quantile_format.sql
Normal file
26
dbms/tests/queries/0_stateless/00753_quantile_format.sql
Normal file
@ -0,0 +1,26 @@
|
||||
CREATE TABLE test.datetime (d DateTime) ENGINE = Memory;
|
||||
INSERT INTO test.datetime(d) VALUES(toDateTime('2016-06-15 23:00:00'));
|
||||
|
||||
SELECT quantile(0.2)(d) FROM test.datetime;
|
||||
SELECT quantiles(0.2)(d) FROM test.datetime;
|
||||
|
||||
SELECT quantileDeterministic(0.2)(d, 1) FROM test.datetime;
|
||||
SELECT quantilesDeterministic(0.2)(d, 1) FROM test.datetime;
|
||||
|
||||
SELECT quantileExact(0.2)(d) FROM test.datetime;
|
||||
SELECT quantilesExact(0.2)(d) FROM test.datetime;
|
||||
|
||||
SELECT quantileExactWeighted(0.2)(d, 1) FROM test.datetime;
|
||||
SELECT quantilesExactWeighted(0.2)(d, 1) FROM test.datetime;
|
||||
|
||||
SELECT quantileTiming(0.2)(d) FROM test.datetime;
|
||||
SELECT quantilesTiming(0.2)(d) FROM test.datetime;
|
||||
|
||||
SELECT quantileTimingWeighted(0.2)(d, 1) FROM test.datetime;
|
||||
SELECT quantilesTimingWeighted(0.2)(d, 1) FROM test.datetime;
|
||||
|
||||
SELECT quantileTDigest(0.2)(d) FROM test.datetime;
|
||||
SELECT quantilesTDigest(0.2)(d) FROM test.datetime;
|
||||
|
||||
SELECT quantileTDigestWeighted(0.2)(d, 1) FROM test.datetime;
|
||||
SELECT quantilesTDigestWeighted(0.2)(d, 1) FROM test.datetime;
|
@ -0,0 +1,25 @@
|
||||
┌─name────────────────┬─primary_key─┬─order_key─┬─partition_key─┬─sample_key─┐
|
||||
│ check_system_tables │ name1 │ │ name2 │ name1 │
|
||||
└─────────────────────┴─────────────┴───────────┴───────────────┴────────────┘
|
||||
┌─name──┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐
|
||||
│ name1 │ 1 │ 1 │ 0 │ 1 │
|
||||
│ name2 │ 0 │ 0 │ 1 │ 0 │
|
||||
│ name3 │ 0 │ 0 │ 0 │ 0 │
|
||||
└───────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘
|
||||
┌─name────────────────┬─primary_key─┬─order_key─┬─partition_key─┬─sample_key─┐
|
||||
│ check_system_tables │ date │ version │ date │ │
|
||||
└─────────────────────┴─────────────┴───────────┴───────────────┴────────────┘
|
||||
┌─name────┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐
|
||||
│ date │ 1 │ 1 │ 1 │ 0 │
|
||||
│ value │ 0 │ 0 │ 0 │ 0 │
|
||||
│ version │ 0 │ 1 │ 0 │ 0 │
|
||||
│ sign │ 0 │ 0 │ 0 │ 0 │
|
||||
└─────────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘
|
||||
┌─name────────────────┬─primary_key───────────────────────┬─order_key─┬─partition_key───┬─sample_key────────┐
|
||||
│ check_system_tables │ Counter, Event, intHash32(UserId) │ │ toYYYYMM(Event) │ intHash32(UserId) │
|
||||
└─────────────────────┴───────────────────────────────────┴───────────┴─────────────────┴───────────────────┘
|
||||
┌─name────┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐
|
||||
│ Event │ 1 │ 1 │ 1 │ 0 │
|
||||
│ UserId │ 0 │ 0 │ 0 │ 1 │
|
||||
│ Counter │ 1 │ 1 │ 0 │ 0 │
|
||||
└─────────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘
|
@ -0,0 +1,68 @@
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.check_system_tables;
|
||||
|
||||
-- Check MergeTree declaration in new format
|
||||
CREATE TABLE test.check_system_tables
|
||||
(
|
||||
name1 UInt8,
|
||||
name2 UInt8,
|
||||
name3 UInt8
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY name1
|
||||
PARTITION BY name2
|
||||
SAMPLE BY name1;
|
||||
|
||||
SELECT name, primary_key, order_key, partition_key, sample_key
|
||||
FROM system.tables
|
||||
WHERE name = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key
|
||||
FROM system.columns
|
||||
WHERE table = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
DROP TABLE IF EXISTS test.check_system_tables;
|
||||
|
||||
-- Check VersionedCollapsingMergeTree
|
||||
CREATE TABLE test.check_system_tables
|
||||
(
|
||||
date Date,
|
||||
value String,
|
||||
version UInt64,
|
||||
sign Int8
|
||||
) ENGINE = VersionedCollapsingMergeTree(sign, version)
|
||||
PARTITION BY date
|
||||
ORDER BY date;
|
||||
|
||||
SELECT name, primary_key, order_key, partition_key, sample_key
|
||||
FROM system.tables
|
||||
WHERE name = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key
|
||||
FROM system.columns
|
||||
WHERE table = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
DROP TABLE IF EXISTS test.check_system_tables;
|
||||
|
||||
-- Check MergeTree declaration in old format
|
||||
CREATE TABLE test.check_system_tables
|
||||
(
|
||||
Event Date,
|
||||
UserId UInt32,
|
||||
Counter UInt32
|
||||
) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192);
|
||||
|
||||
SELECT name, primary_key, order_key, partition_key, sample_key
|
||||
FROM system.tables
|
||||
WHERE name = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key
|
||||
FROM system.columns
|
||||
WHERE table = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
DROP TABLE IF EXISTS test.check_system_tables;
|
@ -1,3 +1,5 @@
|
||||
<a name="http_interface"></a>
|
||||
|
||||
# HTTP Interface
|
||||
|
||||
The HTTP interface lets you use ClickHouse on any platform from any programming language. We use it for working from Java and Perl, as well as shell scripts. In other departments, the HTTP interface is used from Perl, Python, and Go. The HTTP interface is more limited than the native interface, but it has better compatibility.
|
||||
|
53
docs/en/operations/settings/permissions_for_queries.md
Normal file
53
docs/en/operations/settings/permissions_for_queries.md
Normal file
@ -0,0 +1,53 @@
|
||||
<a name="permissions_for_queries"></a>
|
||||
|
||||
# Permissions for queries
|
||||
|
||||
Queries in ClickHouse can be divided into several groups:
|
||||
|
||||
1. Read data queries: `SELECT`, `SHOW`, `DESCRIBE`, `EXISTS`.
|
||||
1. Write data queries: `INSERT`, `OPTIMIZE`.
|
||||
1. Change settings queries: `SET`, `USE`.
|
||||
1. [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries: `CREATE`, `ALTER`, `RENAME`, `ATTACH`, `DETACH`, `DROP` `TRUNCATE`.
|
||||
1. Particular queries: `KILL QUERY`.
|
||||
|
||||
The following settings regulate user permissions for the groups of queries:
|
||||
|
||||
- [readonly](#settings_readonly) — Restricts permissions for all groups of queries excepting DDL.
|
||||
- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries.
|
||||
|
||||
`KILL QUERY` performs with any settings.
|
||||
|
||||
<a name="settings_readonly"></a>
|
||||
|
||||
## readonly
|
||||
|
||||
Restricts permissions for read data, write data and change settings queries.
|
||||
|
||||
See [above](#permissions_for_queries) for the division of queries into groups.
|
||||
|
||||
**Possible values**
|
||||
|
||||
- 0 — All queries are allowed. Default value.
|
||||
- 1 — Read data queries only are allowed.
|
||||
- 2 — Read data and change settings queries are allowed.
|
||||
|
||||
After setting `readonly = 1`, a user can't change `readonly` and `allow_ddl` settings in the current session.
|
||||
|
||||
When using the `GET` method in the [HTTP interface](../../interfaces/http_interface.md#http_interface), `readonly = 1` is set automatically. To modify data use the `POST` method.
|
||||
|
||||
<a name="settings_allow_ddl"></a>
|
||||
|
||||
## allow_ddl
|
||||
|
||||
Allows/denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries.
|
||||
|
||||
See [above](#permissions_for_queries) for the division of queries into groups.
|
||||
|
||||
**Possible values**
|
||||
|
||||
- 0 — DDL queries are not allowed.
|
||||
- 1 — DDL queries are allowed. Default value.
|
||||
|
||||
You can not execute `SET allow_ddl = 1` if `allow_ddl = 0` for current session.
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/operations/settings/permissions_for_queries/) <!--hide-->
|
@ -16,18 +16,6 @@ It can take one of two values: `throw` or `break`. Restrictions on aggregation (
|
||||
|
||||
`any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set.
|
||||
|
||||
<a name="query_complexity_readonly"></a>
|
||||
|
||||
## readonly
|
||||
|
||||
With a value of 0, you can execute any queries.
|
||||
With a value of 1, you can only execute read requests (such as SELECT and SHOW). Requests for writing and changing settings (INSERT, SET) are prohibited.
|
||||
With a value of 2, you can process read queries (SELECT, SHOW) and change settings (SET).
|
||||
|
||||
After enabling readonly mode, you can't disable it in the current session.
|
||||
|
||||
When using the GET method in the HTTP interface, 'readonly = 1' is set automatically. In other words, for queries that modify data, you can only use the POST method. You can send the query itself either in the POST body, or in the URL parameter.
|
||||
|
||||
<a name="settings_max_memory_usage"></a>
|
||||
|
||||
## max_memory_usage
|
||||
|
@ -55,7 +55,7 @@ arrayConcat(arrays)
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `arrays` – arbitrary number of arguments of type Array.
|
||||
- `arrays` – Arbitrary number of arguments of [Array][../../data_types/array.md#data_type-array] type.
|
||||
|
||||
**Example**
|
||||
|
||||
|
1
docs/fa/operations/settings/permissions_for_queries.md
Symbolic link
1
docs/fa/operations/settings/permissions_for_queries.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/operations/settings/permissions_for_queries.md
|
1
docs/ru/operations/settings/permissions_for_queries.md
Symbolic link
1
docs/ru/operations/settings/permissions_for_queries.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/operations/settings/permissions_for_queries.md
|
@ -55,7 +55,7 @@ arrayConcat(arrays)
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `arrays` - Перечисленные через запятую массивы `[values]`.
|
||||
- `arrays` – произвольное количество элементов типа [Array][../../data_types/array.md#data_type-array].
|
||||
|
||||
**Пример**
|
||||
|
||||
|
@ -157,10 +157,10 @@ nav:
|
||||
- 'Server settings': 'operations/server_settings/settings.md'
|
||||
- 'Settings':
|
||||
- 'Introduction': 'operations/settings/index.md'
|
||||
- 'Permissions for queries': 'operations/settings/permissions_for_queries.md'
|
||||
- 'Restrictions on query complexity': 'operations/settings/query_complexity.md'
|
||||
- 'Settings': 'operations/settings/settings.md'
|
||||
- 'Settings profiles': 'operations/settings/settings_profiles.md'
|
||||
|
||||
- 'Utilities':
|
||||
- 'Overview': 'operations/utils/index.md'
|
||||
- 'clickhouse-copier': 'operations/utils/clickhouse-copier.md'
|
||||
|
@ -154,6 +154,7 @@ nav:
|
||||
- 'Server settings': 'operations/server_settings/settings.md'
|
||||
- 'Settings':
|
||||
- 'Introduction': 'operations/settings/index.md'
|
||||
- 'Permissions for queries': 'operations/settings/permissions_for_queries.md'
|
||||
- 'Restrictions on query complexity': 'operations/settings/query_complexity.md'
|
||||
- 'Settings': 'operations/settings/settings.md'
|
||||
- 'Settings profiles': 'operations/settings/settings_profiles.md'
|
||||
|
@ -158,6 +158,7 @@ nav:
|
||||
- 'Серверные настройки': 'operations/server_settings/settings.md'
|
||||
- 'Настройки':
|
||||
- 'Введение': 'operations/settings/index.md'
|
||||
- 'Разрешения на выполнение запросов': 'operations/settings/permissions_for_queries.md'
|
||||
- 'Ограничения на сложность запроса': 'operations/settings/query_complexity.md'
|
||||
- 'Настройки': 'operations/settings/settings.md'
|
||||
- 'Профили настроек': 'operations/settings/settings_profiles.md'
|
||||
|
@ -157,6 +157,7 @@ nav:
|
||||
- 'Server settings': 'operations/server_settings/settings.md'
|
||||
- 'Settings':
|
||||
- 'Introduction': 'operations/settings/index.md'
|
||||
- 'Permissions for queries': 'operations/settings/permissions_for_queries.md'
|
||||
- 'Restrictions on query complexity': 'operations/settings/query_complexity.md'
|
||||
- 'Settings': 'operations/settings/settings.md'
|
||||
- 'Settings profiles': 'operations/settings/settings_profiles.md'
|
||||
|
1
docs/zh/operations/settings/permissions_for_queries.md
Symbolic link
1
docs/zh/operations/settings/permissions_for_queries.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/operations/settings/permissions_for_queries.md
|
Loading…
Reference in New Issue
Block a user