diff --git a/docker/bare/README.md b/docker/bare/README.md index 7b5ab6f5ea9..0a610b1de90 100644 --- a/docker/bare/README.md +++ b/docker/bare/README.md @@ -34,4 +34,5 @@ sudo chroot . /clickhouse server - creation of `clickhouse` user to run the server; - VOLUME for server; -- most of the details, see other docker images for comparison. +- CA Certificates; +- most of the details, see other docker images for comparison; diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index e20fd89aafa..7766a8348af 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -86,9 +86,17 @@ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-clien clickhouse-server --config /etc/clickhouse-server/config.xml --daemon +counter=0 + until clickhouse-client --query "SELECT 1" do sleep 0.1 + if [ "$counter" -gt 1200 ] + then + break + fi + + counter=$(($counter + 1)) done TESTS_TO_SKIP=( @@ -147,8 +155,6 @@ TESTS_TO_SKIP=( 01280_ssd_complex_key_dictionary 00652_replicated_mutations_zookeeper 01411_bayesian_ab_testing - # TRUNCATE TABLE system.query_log -- conflicts with other tests - 01413_rows_events ) clickhouse-test -j 4 --no-long --testname --shard --zookeeper --skip ${TESTS_TO_SKIP[*]} 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt @@ -178,9 +184,16 @@ if [[ ! -z "$FAILED_TESTS" ]]; then clickhouse-server --config /etc/clickhouse-server/config.xml --daemon + counter=0 until clickhouse-client --query "SELECT 1" do sleep 0.1 + if [ "$counter" -gt 1200 ] + then + break + fi + + counter=$(($counter + 1)) done echo "Going to run again: $FAILED_TESTS" diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6669f8dd179..908bbd60c7a 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -29,7 +29,7 @@ echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_module echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment -service clickhouse-server start +timeout 120 service clickhouse-server start wait_server @@ -37,7 +37,9 @@ wait_server chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" -service clickhouse-server restart + +timeout 120 service clickhouse-server stop +timeout 120 service clickhouse-server start wait_server @@ -49,7 +51,8 @@ clickhouse-client --query "SHOW TABLES FROM test" ./stress --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" -service clickhouse-server restart +timeout 120 service clickhouse-server stop +timeout 120 service clickhouse-server start wait_server diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index c42108ee40e..a4378388ef5 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -121,7 +121,7 @@ tickTime=2000 # The number of ticks that the initial # synchronization phase can take # This value is not quite motivated -initLimit=30000 +initLimit=300 # The number of ticks that can pass between # sending a request and getting an acknowledgement syncLimit=10 diff --git a/docs/zh/sql-reference/syntax.md b/docs/zh/sql-reference/syntax.md index a53de43d8f4..a2ce1b5bac3 100644 --- a/docs/zh/sql-reference/syntax.md +++ b/docs/zh/sql-reference/syntax.md @@ -187,9 +187,8 @@ select查询中,星号可以代替表达式使用。详情请参见“select ## 表达式 {#syntax-expressions} - -An expression is a function, identifier, literal, application of an operator, expression in brackets, subquery, or asterisk. It can also contain an alias. -A list of expressions is one or more expressions separated by commas. -Functions and operators, in turn, can have expressions as arguments. +表达式是函数、标识符、字符、运算符的应用程序、括号中的表达式、子查询或星号。它也可以包含别名。 +表达式列表是用逗号分隔的一个或多个表达式。 +反过来,函数和运算符可以将表达式作为参数。 [原始文档](https://clickhouse.tech/docs/en/sql_reference/syntax/) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index bb862348e52..440d080637f 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -232,7 +232,7 @@ private: context.setQueryParameters(query_parameters); /// settings and limits could be specified in config file, but passed settings has higher priority - for (auto setting : context.getSettingsRef().allUnchanged()) + for (const auto & setting : context.getSettingsRef().allUnchanged()) { const auto & name = setting.getName(); if (config().has(name)) @@ -2252,7 +2252,7 @@ public: /// Copy settings-related program options to config. /// TODO: Is this code necessary? - for (auto setting : context.getSettingsRef().all()) + for (const auto & setting : context.getSettingsRef().all()) { const auto & name = setting.getName(); if (options.count(name)) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e619511d3f1..fb6cd38a193 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -223,7 +223,7 @@ void checkForUserSettingsAtTopLevel(const Poco::Util::AbstractConfiguration & co return; Settings settings; - for (auto setting : settings.all()) + for (const auto & setting : settings.all()) { const auto & name = setting.getName(); if (config.has(name)) @@ -644,6 +644,9 @@ int Server::main(const std::vector & /*args*/) global_context->setFormatSchemaPath(format_schema_path.path()); format_schema_path.createDirectories(); + /// Check sanity of MergeTreeSettings on server startup + global_context->getMergeTreeSettings().sanityCheck(settings); + /// Limit on total memory usage size_t max_server_memory_usage = config().getUInt64("max_server_memory_usage", 0); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index f8084e3716f..6a0397837a5 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -23,15 +23,15 @@ namespace template struct MovingSum { - using DataType = MovingSumData; - using Function = MovingImpl; + using Data = MovingSumData, Decimal128, NearestFieldType>>; + using Function = MovingImpl; }; template struct MovingAvg { - using DataType = MovingAvgData; - using Function = MovingImpl; + using Data = MovingAvgData, Decimal128, Float64>>; + using Function = MovingImpl; }; template using MovingSumTemplate = typename MovingSum::Function; @@ -71,11 +71,11 @@ AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, con { auto type = parameters[0].getType(); if (type != Field::Types::Int64 && type != Field::Types::UInt64) - throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Parameter for aggregate function " + name + " should be positive integer", ErrorCodes::BAD_ARGUMENTS); if ((type == Field::Types::Int64 && parameters[0].get() < 0) || (type == Field::Types::UInt64 && parameters[0].get() == 0)) - throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Parameter for aggregate function " + name + " should be positive integer", ErrorCodes::BAD_ARGUMENTS); limit_size = true; max_elems = parameters[0].get(); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index 13895dea8d4..a933c5dde06 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -28,60 +28,51 @@ namespace ErrorCodes extern const int TOO_LARGE_ARRAY_SIZE; } - template -struct MovingSumData +struct MovingData { - // Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena + using Accumulator = T; + + /// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena using Allocator = MixedAlignedArenaAllocator; using Array = PODArray; - Array value; - Array window; + Array value; /// Prefix sums. T sum = 0; void add(T val, Arena * arena) { sum += val; - value.push_back(sum, arena); } - - T get(size_t idx, UInt64 win_size) const - { - if (idx < win_size) - return value[idx]; - else - return value[idx] - value[idx - win_size]; - } - }; template -struct MovingAvgData +struct MovingSumData : public MovingData { - // Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena - using Allocator = MixedAlignedArenaAllocator; - using Array = PODArray; + static constexpr auto name = "groupArrayMovingSum"; - Array value; - Array window; - T sum = 0; - - void add(T val, Arena * arena) + T get(size_t idx, UInt64 window_size) const { - sum += val; - value.push_back(sum, arena); - } - - T get(size_t idx, UInt64 win_size) const - { - if (idx < win_size) - return value[idx] / win_size; + if (idx < window_size) + return this->value[idx]; else - return (value[idx] - value[idx - win_size]) / win_size; + return this->value[idx] - this->value[idx - window_size]; } +}; +template +struct MovingAvgData : public MovingData +{ + static constexpr auto name = "groupArrayMovingAvg"; + + T get(size_t idx, UInt64 window_size) const + { + if (idx < window_size) + return this->value[idx] / window_size; + else + return (this->value[idx] - this->value[idx - window_size]) / window_size; + } }; @@ -90,30 +81,43 @@ class MovingImpl final : public IAggregateFunctionDataHelper> { static constexpr bool limit_num_elems = Tlimit_num_elems::value; - DataTypePtr & data_type; - UInt64 win_size; + UInt64 window_size; public: - using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; - // probably for overflow function in the future - using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + using ResultT = typename Data::Accumulator; - explicit MovingImpl(const DataTypePtr & data_type_, UInt64 win_size_ = std::numeric_limits::max()) + using ColumnSource = std::conditional_t, + ColumnDecimal, + ColumnVector>; + + /// Probably for overflow function in the future. + using ColumnResult = std::conditional_t, + ColumnDecimal, + ColumnVector>; + + using DataTypeResult = std::conditional_t, + DataTypeDecimal, + DataTypeNumber>; + + explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits::max()) : IAggregateFunctionDataHelper>({data_type_}, {}) - , data_type(this->argument_types[0]), win_size(win_size_) {} + , window_size(window_size_) {} - String getName() const override { return "movingXXX"; } + String getName() const override { return Data::name; } DataTypePtr getReturnType() const override { - return std::make_shared(data_type); + if constexpr (IsDecimalNumber) + return std::make_shared(std::make_shared( + DataTypeResult::maxPrecision(), getDecimalScale(*this->argument_types.at(0)))); + else + return std::make_shared(std::make_shared()); } void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - auto val = static_cast(*columns[0]).getData()[row_num]; - - this->data(place).add(val, arena); + auto value = static_cast(*columns[0]).getData()[row_num]; + this->data(place).add(value, arena); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override @@ -170,7 +174,7 @@ public: if (size) { - typename ColVecResult::Container & data_to = static_cast(arr_to.getData()).getData(); + typename ColumnResult::Container & data_to = assert_cast(arr_to.getData()).getData(); for (size_t i = 0; i < size; ++i) { @@ -180,7 +184,7 @@ public: } else { - data_to.push_back(data.get(i, win_size)); + data_to.push_back(data.get(i, window_size)); } } } diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 920232ee92c..8621e0d5db9 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -276,7 +276,8 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).set.insert(typename Data::Set::value_type(UniqVariadicHash::apply(num_args, columns, row_num))); + this->data(place).set.insert(typename Data::Set::value_type( + UniqVariadicHash::apply(num_args, columns, row_num))); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 25d8580a923..a0876d457b8 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -151,7 +151,8 @@ public: virtual void addBatchSinglePlaceNotNull( size_t batch_size, AggregateDataPtr place, const IColumn ** columns, const UInt8 * null_map, Arena * arena) const = 0; - virtual void addBatchSinglePlaceFromInterval(size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; + virtual void addBatchSinglePlaceFromInterval( + size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; /** In addition to addBatch, this method collects multiple rows of arguments into array "places" * as long as they are between offsets[i-1] and offsets[i]. This is used for arrayReduce and @@ -159,7 +160,24 @@ public: * "places" contains a large number of same values consecutively. */ virtual void addBatchArray( - size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena) const = 0; + size_t batch_size, + AggregateDataPtr * places, + size_t place_offset, + const IColumn ** columns, + const UInt64 * offsets, + Arena * arena) const = 0; + + /** The case when the aggregation key is UInt8 + * and pointers to aggregation states are stored in AggregateDataPtr[256] lookup table. + */ + virtual void addBatchLookupTable8( + size_t batch_size, + AggregateDataPtr * places, + size_t place_offset, + std::function init, + const UInt8 * key, + const IColumn ** columns, + Arena * arena) const = 0; /** By default all NULLs are skipped during aggregation. * If it returns nullptr, the default one will be used. @@ -204,6 +222,24 @@ public: static_cast(this)->add(places[i] + place_offset, columns, i, arena); } + void addBatchLookupTable8( + size_t batch_size, + AggregateDataPtr * places, + size_t place_offset, + std::function init, + const UInt8 * key, + const IColumn ** columns, + Arena * arena) const override + { + for (size_t i = 0; i < batch_size; ++i) + { + AggregateDataPtr & place = places[key[i]]; + if (unlikely(!place)) + init(place); + static_cast(this)->add(place + place_offset, columns, i, arena); + } + } + void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override { for (size_t i = 0; i < batch_size; ++i) @@ -218,7 +254,8 @@ public: static_cast(this)->add(place, columns, i, arena); } - void addBatchSinglePlaceFromInterval(size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override + void addBatchSinglePlaceFromInterval( + size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override { for (size_t i = batch_begin; i < batch_end; ++i) static_cast(this)->add(place, columns, i, arena); diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index bc6d7413def..639a0c026ff 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -2,6 +2,7 @@ #include #include +#include /** Returns log2 of number, rounded down. @@ -15,11 +16,15 @@ inline unsigned int bitScanReverse(unsigned int x) /** For zero argument, result is zero. - * For arguments with most significand bit set, result is zero. + * For arguments with most significand bit set, result is n. * For other arguments, returns value, rounded up to power of two. */ inline size_t roundUpToPowerOfTwoOrZero(size_t n) { + // if MSB is set, return n, to avoid return zero + if (unlikely(n >= 0x8000000000000000ULL)) + return n; + --n; n |= n >> 1; n |= n >> 2; diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index 10d28078d58..a7fcfd4f8c0 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -64,6 +64,8 @@ struct HashMethodOneNumber /// Is used for default implementation in HashMethodBase. FieldType getKeyHolder(size_t row, Arena &) const { return unalignedLoad(vec + row * sizeof(FieldType)); } + + const FieldType * getKeyData() const { return reinterpret_cast(vec); } }; diff --git a/src/Common/HashTable/FixedClearableHashSet.h b/src/Common/HashTable/FixedClearableHashSet.h index 32cb6df924a..19d4669831f 100644 --- a/src/Common/HashTable/FixedClearableHashSet.h +++ b/src/Common/HashTable/FixedClearableHashSet.h @@ -34,10 +34,11 @@ struct FixedClearableHashTableCell template -class FixedClearableHashSet : public FixedHashTable, Allocator> +class FixedClearableHashSet : public FixedHashTable< + Key, FixedClearableHashTableCell, FixedHashTableStoredSize>, Allocator> { public: - using Base = FixedHashTable, Allocator>; + using Base = FixedHashTable, FixedHashTableStoredSize>, Allocator>; using LookupResult = typename Base::LookupResult; void clear() diff --git a/src/Common/HashTable/FixedHashMap.h b/src/Common/HashTable/FixedHashMap.h index 45f5c9f12e2..9fc331e09e4 100644 --- a/src/Common/HashTable/FixedHashMap.h +++ b/src/Common/HashTable/FixedHashMap.h @@ -94,11 +94,16 @@ struct FixedHashMapImplicitZeroCell }; -template , typename Allocator = HashTableAllocator> -class FixedHashMap : public FixedHashTable +template < + typename Key, + typename Mapped, + typename Cell = FixedHashMapCell, + typename Size = FixedHashTableStoredSize, + typename Allocator = HashTableAllocator> +class FixedHashMap : public FixedHashTable { public: - using Base = FixedHashTable; + using Base = FixedHashTable; using Self = FixedHashMap; using LookupResult = typename Base::LookupResult; @@ -155,5 +160,19 @@ public: } }; -template , typename Allocator = HashTableAllocator> -using FixedImplicitZeroHashMap = FixedHashMap; + +template +using FixedImplicitZeroHashMap = FixedHashMap< + Key, + Mapped, + FixedHashMapImplicitZeroCell, + FixedHashTableStoredSize>, + Allocator>; + +template +using FixedImplicitZeroHashMapWithCalculatedSize = FixedHashMap< + Key, + Mapped, + FixedHashMapImplicitZeroCell, + FixedHashTableCalculatedSize>, + Allocator>; diff --git a/src/Common/HashTable/FixedHashSet.h b/src/Common/HashTable/FixedHashSet.h index ce3666944dd..e764038e6c3 100644 --- a/src/Common/HashTable/FixedHashSet.h +++ b/src/Common/HashTable/FixedHashSet.h @@ -3,11 +3,11 @@ #include template -class FixedHashSet : public FixedHashTable, Allocator> +class FixedHashSet : public FixedHashTable, FixedHashTableStoredSize>, Allocator> { public: using Cell = FixedHashTableCell; - using Base = FixedHashTable; + using Base = FixedHashTable, Allocator>; using Self = FixedHashSet; void merge(const Self & rhs) diff --git a/src/Common/HashTable/FixedHashTable.h b/src/Common/HashTable/FixedHashTable.h index 0349c4b095f..9d18f03a30b 100644 --- a/src/Common/HashTable/FixedHashTable.h +++ b/src/Common/HashTable/FixedHashTable.h @@ -47,6 +47,47 @@ struct FixedHashTableCell }; +/// How to obtain the size of the table. + +template +struct FixedHashTableStoredSize +{ + size_t m_size = 0; + + size_t getSize(const Cell *, const typename Cell::State &, size_t) const { return m_size; } + bool isEmpty(const Cell *, const typename Cell::State &, size_t) const { return m_size == 0; } + + void increaseSize() { ++m_size; } + void clearSize() { m_size = 0; } + void setSize(size_t to) { m_size = to; } +}; + +template +struct FixedHashTableCalculatedSize +{ + size_t getSize(const Cell * buf, const typename Cell::State & state, size_t num_cells) const + { + size_t res = 0; + for (const Cell * end = buf + num_cells; buf != end; ++buf) + if (!buf->isZero(state)) + ++res; + return res; + } + + bool isEmpty(const Cell * buf, const typename Cell::State & state, size_t num_cells) const + { + for (const Cell * end = buf + num_cells; buf != end; ++buf) + if (!buf->isZero(state)) + return false; + return true; + } + + void increaseSize() {} + void clearSize() {} + void setSize(size_t) {} +}; + + /** Used as a lookup table for small keys such as UInt8, UInt16. It's different * than a HashTable in that keys are not stored in the Cell buf, but inferred * inside each iterator. There are a bunch of to make it faster than using @@ -63,8 +104,8 @@ struct FixedHashTableCell * transfer, key updates (f.g. StringRef) and serde. This will allow * TwoLevelHashSet(Map) to contain different type of sets(maps). */ -template -class FixedHashTable : private boost::noncopyable, protected Allocator, protected Cell::State +template +class FixedHashTable : private boost::noncopyable, protected Allocator, protected Cell::State, protected Size { static constexpr size_t NUM_CELLS = 1ULL << (sizeof(Key) * 8); @@ -75,7 +116,6 @@ protected: using Self = FixedHashTable; - size_t m_size = 0; /// Amount of elements Cell * buf; /// A piece of memory for all elements. void alloc() { buf = reinterpret_cast(Allocator::alloc(NUM_CELLS * sizeof(Cell))); } @@ -178,7 +218,7 @@ public: free(); std::swap(buf, rhs.buf); - std::swap(m_size, rhs.m_size); + this->setSize(rhs.size()); Allocator::operator=(std::move(rhs)); Cell::State::operator=(std::move(rhs)); @@ -305,7 +345,7 @@ public: new (&buf[x]) Cell(x, *this); inserted = true; - ++m_size; + this->increaseSize(); } std::pair ALWAYS_INLINE insert(const value_type & x) @@ -335,7 +375,7 @@ public: void write(DB::WriteBuffer & wb) const { Cell::State::write(wb); - DB::writeVarUInt(m_size, wb); + DB::writeVarUInt(size(), wb); if (!buf) return; @@ -353,7 +393,7 @@ public: void writeText(DB::WriteBuffer & wb) const { Cell::State::writeText(wb); - DB::writeText(m_size, wb); + DB::writeText(size(), wb); if (!buf) return; @@ -374,7 +414,9 @@ public: { Cell::State::read(rb); destroyElements(); + size_t m_size; DB::readVarUInt(m_size, rb); + this->setSize(m_size); free(); alloc(); @@ -392,7 +434,9 @@ public: { Cell::State::readText(rb); destroyElements(); + size_t m_size; DB::readText(m_size, rb); + this->setSize(m_size); free(); alloc(); @@ -408,14 +452,13 @@ public: } } - size_t size() const { return m_size; } - - bool empty() const { return 0 == m_size; } + size_t size() const { return this->getSize(buf, *this, NUM_CELLS); } + bool empty() const { return this->isEmpty(buf, *this, NUM_CELLS); } void clear() { destroyElements(); - m_size = 0; + this->clearSize(); memset(static_cast(buf), 0, NUM_CELLS * sizeof(*buf)); } @@ -425,7 +468,7 @@ public: void clearAndShrink() { destroyElements(); - m_size = 0; + this->clearSize(); free(); } @@ -433,6 +476,9 @@ public: size_t getBufferSizeInCells() const { return NUM_CELLS; } + const Cell * data() const { return buf; } + Cell * data() { return buf; } + #ifdef DBMS_HASH_MAP_COUNT_COLLISIONS size_t getCollisions() const { return 0; } #endif diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index 3ecbd9f263c..e09f60c4294 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -61,9 +61,9 @@ struct HashMapCell /// Get the key (internally). static const Key & getKey(const value_type & value) { return value.first; } - bool keyEquals(const Key & key_) const { return value.first == key_; } - bool keyEquals(const Key & key_, size_t /*hash_*/) const { return value.first == key_; } - bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return value.first == key_; } + bool keyEquals(const Key & key_) const { return bitEquals(value.first, key_); } + bool keyEquals(const Key & key_, size_t /*hash_*/) const { return bitEquals(value.first, key_); } + bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return bitEquals(value.first, key_); } void setHash(size_t /*hash_value*/) {} size_t getHash(const Hash & hash) const { return hash(value.first); } @@ -120,8 +120,8 @@ struct HashMapCellWithSavedHash : public HashMapCell using Base::Base; - bool keyEquals(const Key & key_) const { return this->value.first == key_; } - bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->value.first == key_; } + bool keyEquals(const Key & key_) const { return bitEquals(this->value.first, key_); } + bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && bitEquals(this->value.first, key_); } bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); } void setHash(size_t hash_value) { saved_hash = hash_value; } diff --git a/src/Common/HashTable/HashSet.h b/src/Common/HashTable/HashSet.h index c79e05073fc..19dba189ddc 100644 --- a/src/Common/HashTable/HashSet.h +++ b/src/Common/HashTable/HashSet.h @@ -76,8 +76,8 @@ struct HashSetCellWithSavedHash : public HashTableCell HashSetCellWithSavedHash() : Base() {} HashSetCellWithSavedHash(const Key & key_, const typename Base::State & state) : Base(key_, state) {} - bool keyEquals(const Key & key_) const { return this->key == key_; } - bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->key == key_; } + bool keyEquals(const Key & key_) const { return bitEquals(this->key, key_); } + bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && bitEquals(this->key, key_); } bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); } void setHash(size_t hash_value) { saved_hash = hash_value; } diff --git a/src/Common/HashTable/HashTable.h b/src/Common/HashTable/HashTable.h index 528e719c05b..5c8e7917eb0 100644 --- a/src/Common/HashTable/HashTable.h +++ b/src/Common/HashTable/HashTable.h @@ -75,6 +75,25 @@ void set(T & x) { x = 0; } } + +/** Numbers are compared bitwise. + * Complex types are compared by operator== as usual (this is important if there are gaps). + * + * This is needed if you use floats as keys. They are compared by bit equality. + * Otherwise the invariants in hash table probing do not met when NaNs are present. + */ +template +inline bool bitEquals(T && a, T && b) +{ + using RealT = std::decay_t; + + if constexpr (std::is_floating_point_v) + return 0 == memcmp(&a, &b, sizeof(RealT)); /// Note that memcmp with constant size is compiler builtin. + else + return a == b; +} + + /** * getKey/Mapped -- methods to get key/"mapped" values from the LookupResult returned by find() and * emplace() methods of HashTable. Must not be called for a null LookupResult. @@ -150,9 +169,9 @@ struct HashTableCell static const Key & getKey(const value_type & value) { return value; } /// Are the keys at the cells equal? - bool keyEquals(const Key & key_) const { return key == key_; } - bool keyEquals(const Key & key_, size_t /*hash_*/) const { return key == key_; } - bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return key == key_; } + bool keyEquals(const Key & key_) const { return bitEquals(key, key_); } + bool keyEquals(const Key & key_, size_t /*hash_*/) const { return bitEquals(key, key_); } + bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return bitEquals(key, key_); } /// If the cell can remember the value of the hash function, then remember it. void setHash(size_t /*hash_value*/) {} diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index cb257cf95a4..7b5a896015b 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -22,7 +22,11 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & } } -String Macros::expand(const String & s, size_t level, const String & database_name, const String & table_name) const +String Macros::expand(const String & s, + size_t level, + const String & database_name, + const String & table_name, + const UUID & uuid) const { if (s.find('{') == String::npos) return s; @@ -64,10 +68,12 @@ String Macros::expand(const String & s, size_t level, const String & database_na res += database_name; else if (macro_name == "table" && !table_name.empty()) res += table_name; + else if (macro_name == "uuid" && uuid != UUIDHelpers::Nil) + res += toString(uuid); else throw Exception("No macro '" + macro_name + - "' in config while processing substitutions in '" + s + "' at " - + toString(begin), ErrorCodes::SYNTAX_ERROR); + "' in config while processing substitutions in '" + s + "' at '" + + toString(begin) + "' or macro is not supported here", ErrorCodes::SYNTAX_ERROR); pos = end + 1; } @@ -82,9 +88,9 @@ String Macros::getValue(const String & key) const throw Exception("No macro " + key + " in config", ErrorCodes::SYNTAX_ERROR); } -String Macros::expand(const String & s, const String & database_name, const String & table_name) const +String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const { - return expand(s, 0, database_name, table_name); + return expand(s, 0, table_id.database_name, table_id.table_name, allow_uuid ? table_id.uuid : UUIDHelpers::Nil); } Names Macros::expand(const Names & source_names, size_t level) const diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 3409cf542b8..cee133b0ccb 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -30,9 +31,13 @@ public: * If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively. * level - the level of recursion. */ - String expand(const String & s, size_t level = 0, const String & database_name = "", const String & table_name = "") const; + String expand(const String & s, + size_t level = 0, + const String & database_name = "", + const String & table_name = "", + const UUID & uuid = UUIDHelpers::Nil) const; - String expand(const String & s, const String & database_name, const String & table_name) const; + String expand(const String & s, const StorageID & table_id, bool allow_uuid) const; /** Apply expand for the list. diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 75a76340a4f..7de87b345c1 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -104,13 +104,10 @@ public: private: friend class BaseSettings; - SettingFieldRef(const typename Traits::Data & data_, const typename Traits::Accessor & accessor_, size_t index_) : data(&data_), accessor(&accessor_), index(index_) {} - SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_); - - const typename Traits::Data * data = nullptr; - const typename Traits::Accessor * accessor = nullptr; - size_t index = 0; - std::conditional_t custom_setting = {}; + const BaseSettings * settings; + const typename Traits::Accessor * accessor; + size_t index; + std::conditional_t custom_setting; }; enum SkipFlags @@ -128,7 +125,7 @@ public: public: Iterator & operator++(); Iterator operator++(int); - SettingFieldRef operator *() const; + const SettingFieldRef & operator *() const { return field_ref; } bool operator ==(const Iterator & other) const; bool operator !=(const Iterator & other) const { return !(*this == other); } @@ -137,10 +134,9 @@ public: friend class BaseSettings; Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_); void doSkip(); + void setPointerToCustomSetting(); - const BaseSettings * settings = nullptr; - const typename Traits::Accessor * accessor = nullptr; - size_t index; + SettingFieldRef field_ref; std::conditional_t custom_settings_iterator; SkipFlags skip_flags; }; @@ -557,13 +553,20 @@ const SettingFieldCustom * BaseSettings::tryGetCustomSetting(const std: template BaseSettings::Iterator::Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_) - : settings(&settings_), accessor(&accessor_), skip_flags(skip_flags_) + : skip_flags(skip_flags_) { + field_ref.settings = &settings_; + field_ref.accessor = &accessor_; + auto & index = field_ref.index; + if (skip_flags == SKIP_ALL) { - index = accessor->size(); + index = accessor_.size(); if constexpr (Traits::allow_custom_settings) - custom_settings_iterator = settings->custom_settings_map.end(); + { + custom_settings_iterator = settings_.custom_settings_map.end(); + field_ref.custom_setting = nullptr; + } return; } @@ -575,25 +578,28 @@ BaseSettings::Iterator::Iterator(const BaseSettings & settings_, const } if (skip_flags & SKIP_BUILTIN) - index = accessor->size(); + index = accessor_.size(); else index = 0; if constexpr (Traits::allow_custom_settings) { if (skip_flags & SKIP_CUSTOM) - custom_settings_iterator = settings->custom_settings_map.end(); + custom_settings_iterator = settings_.custom_settings_map.end(); else - custom_settings_iterator = settings->custom_settings_map.begin(); + custom_settings_iterator = settings_.custom_settings_map.begin(); } doSkip(); + setPointerToCustomSetting(); } template typename BaseSettings::Iterator & BaseSettings::Iterator::operator++() { - if (index != accessor->size()) + const auto & accessor = *field_ref.accessor; + auto & index = field_ref.index; + if (index != accessor.size()) ++index; else { @@ -601,6 +607,7 @@ typename BaseSettings::Iterator & BaseSettings::Iterator::oper ++custom_settings_iterator; } doSkip(); + setPointerToCustomSetting(); return *this; } @@ -612,32 +619,39 @@ typename BaseSettings::Iterator BaseSettings::Iterator::operat return res; } -template -typename BaseSettings::SettingFieldRef BaseSettings::Iterator::operator*() const -{ - if constexpr (Traits::allow_custom_settings) - { - if (index == accessor->size()) - return {custom_settings_iterator->second}; - } - return {*settings, *accessor, index}; -} - template void BaseSettings::Iterator::doSkip() { + const auto & accessor = *field_ref.accessor; + const auto & settings = *field_ref.settings; + auto & index = field_ref.index; if (skip_flags & SKIP_CHANGED) { - while ((index != accessor->size()) && accessor->isValueChanged(*settings, index)) + while ((index != accessor.size()) && accessor.isValueChanged(settings, index)) ++index; } else if (skip_flags & SKIP_UNCHANGED) { - while ((index != accessor->size()) && !accessor->isValueChanged(*settings, index)) + while ((index != accessor.size()) && !accessor.isValueChanged(settings, index)) ++index; } } +template +void BaseSettings::Iterator::setPointerToCustomSetting() +{ + if constexpr (Traits::allow_custom_settings) + { + const auto & accessor = *field_ref.accessor; + const auto & settings = *field_ref.settings; + const auto & index = field_ref.index; + if ((index == accessor.size()) && (custom_settings_iterator != settings.custom_settings_map.end())) + field_ref.custom_setting = &custom_settings_iterator->second; + else + field_ref.custom_setting = nullptr; + } +} + template bool BaseSettings::Iterator::operator ==(const typename BaseSettings::Iterator & other) const { @@ -646,14 +660,7 @@ bool BaseSettings::Iterator::operator ==(const typename BaseSettings -BaseSettings::SettingFieldRef::SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_) -{ - if constexpr (Traits_::allow_custom_settings) - custom_setting = &custom_setting_; + return ((field_ref.index == other.field_ref.index) && (field_ref.settings == other.field_ref.settings)); } template @@ -675,7 +682,7 @@ Field BaseSettings::SettingFieldRef::getValue() const if (custom_setting) return static_cast(custom_setting->second); } - return accessor->getValue(*data, index); + return accessor->getValue(*settings, index); } template @@ -686,7 +693,7 @@ String BaseSettings::SettingFieldRef::getValueString() const if (custom_setting) return custom_setting->second.toString(); } - return accessor->getValueString(*data, index); + return accessor->getValueString(*settings, index); } template @@ -697,7 +704,7 @@ bool BaseSettings::SettingFieldRef::isValueChanged() const if (custom_setting) return true; } - return accessor->isValueChanged(*data, index); + return accessor->isValueChanged(*settings, index); } template diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 7060ecb4bda..f988e24d9f5 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -65,7 +65,7 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu size_t count = 0; - for (auto setting : all(changed_only ? SKIP_UNCHANGED : SKIP_NONE)) + for (const auto & setting : all(changed_only ? SKIP_UNCHANGED : SKIP_NONE)) { if (column_names) { @@ -95,7 +95,7 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu void Settings::addProgramOptions(boost::program_options::options_description & options) { - for (auto field : all()) + for (const auto & field : all()) { const std::string_view name = field.getName(); auto on_program_option diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 07fdeaabf22..ff137a54381 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -347,8 +347,8 @@ class IColumn; M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ - M(Bool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \ - M(Bool, show_table_uuid_in_table_create_query_if_not_nil, true, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ + M(Bool, allow_experimental_database_atomic, true, "Allow to create database with Engine=Atomic.", 0) \ + M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ff30b95d139..1223f9a1d56 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -16,10 +17,13 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_DATABASE; extern const int TABLE_ALREADY_EXISTS; extern const int CANNOT_ASSIGN_ALTER; extern const int DATABASE_NOT_EMPTY; extern const int NOT_IMPLEMENTED; + extern const int FILE_ALREADY_EXISTS; + extern const int INCORRECT_QUERY; } class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator @@ -31,13 +35,15 @@ public: }; -DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_) +DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_) : DatabaseOrdinary(name_, std::move(metadata_path_), "store/", "DatabaseAtomic (" + name_ + ")", context_) - , path_to_table_symlinks(context_.getPath() + "data/" + escapeForFileName(name_) + "/") + , path_to_table_symlinks(global_context.getPath() + "data/" + escapeForFileName(name_) + "/") + , path_to_metadata_symlink(global_context.getPath() + "metadata/" + escapeForFileName(name_)) + , db_uuid(uuid) { - /// Symlinks in data/db_name/ directory are not used by ClickHouse, - /// it's needed only for convenient introspection. + assert(db_uuid != UUIDHelpers::Nil); Poco::File(path_to_table_symlinks).createDirectories(); + tryCreateMetadataSymlink(); } String DatabaseAtomic::getTableDataPath(const String & table_name) const @@ -45,7 +51,7 @@ String DatabaseAtomic::getTableDataPath(const String & table_name) const std::lock_guard lock(mutex); auto it = table_name_to_path.find(table_name); if (it == table_name_to_path.end()) - throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE); + throw Exception("Table " + table_name + " not found in database " + database_name, ErrorCodes::UNKNOWN_TABLE); assert(it->second != data_path && !it->second.empty()); return it->second; } @@ -59,7 +65,15 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const void DatabaseAtomic::drop(const Context &) { - Poco::File(path_to_table_symlinks).remove(true); + try + { + Poco::File(path_to_metadata_symlink).remove(); + Poco::File(path_to_table_symlinks).remove(true); + } + catch (...) + { + LOG_WARNING(log, getCurrentExceptionMessage(true)); + } Poco::File(getMetadataPath()).remove(true); } @@ -69,10 +83,10 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, DetachedTables not_in_use; std::unique_lock lock(mutex); not_in_use = cleenupDetachedTables(); - assertDetachedTableNotInUse(table->getStorageID().uuid); + auto table_id = table->getStorageID(); + assertDetachedTableNotInUse(table_id.uuid); DatabaseWithDictionaries::attachTableUnlocked(name, table, lock); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); - tryCreateSymlink(name, relative_table_path); } StoragePtr DatabaseAtomic::detachTable(const String & name) @@ -83,7 +97,6 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) table_name_to_path.erase(name); detached_tables.emplace(table->getStorageID().uuid, table); not_in_use = cleenupDetachedTables(); - tryRemoveSymlink(name); return table; } @@ -107,16 +120,20 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool } void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database, - const String & to_table_name, bool exchange) + const String & to_table_name, bool exchange, bool dictionary) { if (typeid(*this) != typeid(to_database)) { if (!typeid_cast(&to_database)) throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED); /// Allow moving tables between Atomic and Ordinary (with table lock) - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange); + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary); return; } + + if (exchange && dictionary) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries"); + auto & other_db = dynamic_cast(to_database); bool inside_database = this == &other_db; @@ -125,16 +142,24 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n auto detach = [](DatabaseAtomic & db, const String & table_name_) { - auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second; + auto it = db.table_name_to_path.find(table_name_); + String table_data_path_saved; + /// Path can be not set for DDL dictionaries, but it does not matter for StorageDictionary. + if (it != db.table_name_to_path.end()) + table_data_path_saved = it->second; + assert(!table_data_path_saved.empty() || db.dictionaries.find(table_name_) != db.dictionaries.end()); db.tables.erase(table_name_); db.table_name_to_path.erase(table_name_); - db.tryRemoveSymlink(table_name_); + if (!table_data_path_saved.empty()) + db.tryRemoveSymlink(table_name_); return table_data_path_saved; }; auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_) { db.tables.emplace(table_name_, table_); + if (table_data_path_.empty()) + return; db.table_name_to_path.emplace(table_name_, table_data_path_); db.tryCreateSymlink(table_name_, table_data_path_); }; @@ -169,6 +194,17 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n db_lock = std::unique_lock{mutex}; } + bool is_dictionary = dictionaries.find(table_name) != dictionaries.end(); + if (exchange && other_db.dictionaries.find(to_table_name) != other_db.dictionaries.end()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries"); + + if (dictionary != is_dictionary) + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Use RENAME DICTIONARY for dictionaries and RENAME TABLE for tables."); + + if (is_dictionary && !inside_database) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database"); + StoragePtr table = getTableUnlocked(table_name, db_lock); assert_can_move_mat_view(table); StoragePtr other_table; @@ -189,13 +225,15 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n if (exchange) other_table_data_path = detach(other_db, to_table_name); - table->renameInMemory({other_db.getDatabaseName(), to_table_name, table->getStorageID().uuid}); + auto old_table_id = table->getStorageID(); + + table->renameInMemory({other_db.database_name, to_table_name, old_table_id.uuid}); if (exchange) - other_table->renameInMemory({getDatabaseName(), table_name, other_table->getStorageID().uuid}); + other_table->renameInMemory({database_name, table_name, other_table->getStorageID().uuid}); if (!inside_database) { - DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, other_db.shared_from_this(), table); + DatabaseCatalog::instance().updateUUIDMapping(old_table_id.uuid, other_db.shared_from_this(), table); if (exchange) DatabaseCatalog::instance().updateUUIDMapping(other_table->getStorageID().uuid, shared_from_this(), other_table); } @@ -203,6 +241,12 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n attach(other_db, to_table_name, table_data_path, table); if (exchange) attach(*this, table_name, other_table_data_path, other_table); + + if (is_dictionary) + { + auto new_table_id = StorageID(other_db.database_name, to_table_name, old_table_id.uuid); + renameDictionaryInMemoryUnlocked(old_table_id, new_table_id); + } } void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, @@ -213,6 +257,9 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora try { std::unique_lock lock{mutex}; + if (query.database != database_name) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`", + database_name, query.database); not_in_use = cleenupDetachedTables(); assertDetachedTableNotInUse(query.uuid); renameNoReplace(table_metadata_tmp_path, table_metadata_path); @@ -229,7 +276,8 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) { - SCOPE_EXIT({ std::error_code code; std::filesystem::remove(table_metadata_tmp_path, code); }); + bool check_file_exists = supportsRenameat2(); + SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); }); std::unique_lock lock{mutex}; auto actual_table_id = getTableUnlocked(table_id.table_name, lock)->getStorageID(); @@ -237,7 +285,10 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & if (table_id.uuid != actual_table_id.uuid) throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER); - renameExchange(table_metadata_tmp_path, table_metadata_path); + if (check_file_exists) + renameExchange(table_metadata_tmp_path, table_metadata_path); + else + std::filesystem::rename(table_metadata_tmp_path, table_metadata_path); } void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid) @@ -330,7 +381,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & } catch (...) { - tryLogCurrentException(log); + LOG_WARNING(log, getCurrentExceptionMessage(true)); } } @@ -343,9 +394,108 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name) } catch (...) { - tryLogCurrentException(log); + LOG_WARNING(log, getCurrentExceptionMessage(true)); } } +void DatabaseAtomic::tryCreateMetadataSymlink() +{ + /// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse, + /// it's needed only for convenient introspection. + assert(path_to_metadata_symlink != metadata_path); + Poco::File metadata_symlink(path_to_metadata_symlink); + if (metadata_symlink.exists()) + { + if (!metadata_symlink.isLink()) + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink); + } + else + { + try + { + Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC); + } + catch (...) + { + tryLogCurrentException(log); + } + } +} + +void DatabaseAtomic::renameDatabase(const String & new_name) +{ + /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard + try + { + Poco::File(path_to_metadata_symlink).remove(); + } + catch (...) + { + LOG_WARNING(log, getCurrentExceptionMessage(true)); + } + + auto new_name_escaped = escapeForFileName(new_name); + auto old_database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql"; + auto new_database_metadata_path = global_context.getPath() + "metadata/" + new_name_escaped + ".sql"; + renameNoReplace(old_database_metadata_path, new_database_metadata_path); + + String old_path_to_table_symlinks; + + { + std::lock_guard lock(mutex); + DatabaseCatalog::instance().updateDatabaseName(database_name, new_name); + database_name = new_name; + + for (auto & table : tables) + { + auto table_id = table.second->getStorageID(); + table_id.database_name = database_name; + table.second->renameInMemory(table_id); + } + + for (auto & dict : dictionaries) + { + auto old_name = StorageID(dict.second.create_query); + auto name = old_name; + name.database_name = database_name; + renameDictionaryInMemoryUnlocked(old_name, name); + } + + path_to_metadata_symlink = global_context.getPath() + "metadata/" + new_name_escaped; + old_path_to_table_symlinks = path_to_table_symlinks; + path_to_table_symlinks = global_context.getPath() + "data/" + new_name_escaped + "/"; + } + + Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks); + tryCreateMetadataSymlink(); +} + +void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name) +{ + auto it = dictionaries.find(old_name.table_name); + assert(it != dictionaries.end()); + assert(it->second.config->getString("dictionary.uuid") == toString(old_name.uuid)); + assert(old_name.uuid == new_name.uuid); + it->second.config->setString("dictionary.database", new_name.database_name); + it->second.config->setString("dictionary.name", new_name.table_name); + auto & create = it->second.create_query->as(); + create.database = new_name.database_name; + create.table = new_name.table_name; + assert(create.uuid == new_name.uuid); + + if (old_name.table_name != new_name.table_name) + { + auto attach_info = std::move(it->second); + dictionaries.erase(it); + dictionaries.emplace(new_name.table_name, std::move(attach_info)); + } + + auto result = external_loader.getLoadResult(toString(old_name.uuid)); + if (!result.object) + return; + const auto & dict = dynamic_cast(*result.object); + dict.updateDictionaryName(new_name); +} + } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 71428fdb420..f809fcefef3 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -21,16 +21,20 @@ class DatabaseAtomic : public DatabaseOrdinary { public: - DatabaseAtomic(String name_, String metadata_path_, Context & context_); + DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_); String getEngineName() const override { return "Atomic"; } + UUID getUUID() const override { return db_uuid; } + + void renameDatabase(const String & new_name) override; void renameTable( const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, - bool exchange) override; + bool exchange, + bool dictionary) override; void dropTable(const Context & context, const String & table_name, bool no_delay) override; @@ -51,6 +55,9 @@ public: UUID tryGetTableUUID(const String & table_name) const override; + void tryCreateSymlink(const String & table_name, const String & actual_data_path); + void tryRemoveSymlink(const String & table_name); + private: void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, @@ -60,15 +67,18 @@ private: typedef std::unordered_map DetachedTables; [[nodiscard]] DetachedTables cleenupDetachedTables(); - void tryCreateSymlink(const String & table_name, const String & actual_data_path); - void tryRemoveSymlink(const String & table_name); + void tryCreateMetadataSymlink(); + + void renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name); //TODO store path in DatabaseWithOwnTables::tables typedef std::unordered_map NameToPathMap; NameToPathMap table_name_to_path; DetachedTables detached_tables; - const String path_to_table_symlinks; + String path_to_table_symlinks; + String path_to_metadata_symlink; + const UUID db_uuid; }; } diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 986e36de8cf..3732139c66a 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -55,9 +55,10 @@ Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_nam { Tables tables; auto load_results = global_context.getExternalDictionariesLoader().getLoadResults(filter_by_name); + String db_name = getDatabaseName(); for (auto & load_result : load_results) { - auto storage = createStorageDictionary(getDatabaseName(), load_result); + auto storage = createStorageDictionary(db_name, load_result); if (storage) tables.emplace(storage->getStorageID().table_name, storage); } @@ -77,7 +78,7 @@ StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, const Cont DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { - return std::make_unique(listTables(filter_by_table_name)); + return std::make_unique(listTables(filter_by_table_name), getDatabaseName()); } bool DatabaseDictionary::empty() const @@ -100,7 +101,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, co } auto names_and_types = StorageDictionary::getNamesAndTypes(ExternalDictionariesLoader::getDictionaryStructure(*load_result.config)); - buffer << "CREATE TABLE " << backQuoteIfNeed(database_name) << '.' << backQuoteIfNeed(table_name) << " ("; + buffer << "CREATE TABLE " << backQuoteIfNeed(getDatabaseName()) << '.' << backQuoteIfNeed(table_name) << " ("; buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types); buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")"; } @@ -123,7 +124,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const String query; { WriteBufferFromString buffer(query); - buffer << "CREATE DATABASE " << backQuoteIfNeed(database_name) << " ENGINE = Dictionary"; + buffer << "CREATE DATABASE " << backQuoteIfNeed(getDatabaseName()) << " ENGINE = Dictionary"; } auto settings = global_context.getSettingsRef(); ParserCreateQuery parser; diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index b61c85033e8..c3c6a53a894 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -47,8 +47,6 @@ protected: ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; private: - mutable std::mutex mutex; - Poco::Logger * log; const Context & global_context; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index f27bc509ebe..3b84dfa8949 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -12,6 +12,7 @@ #include #include "DatabaseFactory.h" #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -34,15 +35,19 @@ namespace ErrorCodes extern const int CANNOT_CREATE_DATABASE; } -DatabasePtr DatabaseFactory::get( - const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context) +DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, Context & context) { bool created = false; try { + /// Creates store/xxx/ for Atomic + Poco::File(Poco::Path(metadata_path).makeParent()).createDirectories(); + /// Before 20.7 it's possible that .sql metadata file does not exist for some old database. + /// In this case Ordinary database is created on server startup if the corresponding metadata directory exists. + /// So we should remove metadata directory if database creation failed. created = Poco::File(metadata_path).createDirectory(); - return getImpl(database_name, metadata_path, engine_define, context); + return getImpl(create, metadata_path, context); } catch (...) { @@ -64,10 +69,12 @@ static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &eng return ast->as()->value.safeGet(); } -DatabasePtr DatabaseFactory::getImpl( - const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context) +DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context) { - String engine_name = engine_define->engine->name; + const auto * engine_define = create.storage; + const String & database_name = create.database; + const String & engine_name = engine_define->engine->name; + const UUID & uuid = create.uuid; if (engine_name != "MySQL" && engine_name != "Lazy" && engine_define->engine->arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); @@ -80,7 +87,7 @@ DatabasePtr DatabaseFactory::getImpl( if (engine_name == "Ordinary") return std::make_shared(database_name, metadata_path, context); else if (engine_name == "Atomic") - return std::make_shared(database_name, metadata_path, context); + return std::make_shared(database_name, metadata_path, uuid, context); else if (engine_name == "Memory") return std::make_shared(database_name, context); else if (engine_name == "Dictionary") diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 7df8ee8ada0..88d33dc1cd5 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -6,14 +6,14 @@ namespace DB { -class ASTStorage; +class ASTCreateQuery; class DatabaseFactory { public: - static DatabasePtr get(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context); + static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, Context & context); - static DatabasePtr getImpl(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context); + static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context); }; } diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 11e5272110e..a4be82690e4 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -78,10 +78,11 @@ void DatabaseLazy::renameTable( const String & table_name, IDatabase & to_database, const String & to_table_name, - bool exchange) + bool exchange, + bool dictionary) { SCOPE_EXIT({ clearExpiredTables(); }); - DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange); + DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary); } @@ -91,7 +92,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name auto it = tables_cache.find(table_name); if (it != tables_cache.end()) return it->second.metadata_modification_time; - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); } void DatabaseLazy::alterTable( @@ -160,7 +161,7 @@ void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & tab std::forward_as_tuple(table_name), std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(table_name))); if (!inserted) - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); } @@ -173,7 +174,7 @@ StoragePtr DatabaseLazy::detachTable(const String & table_name) std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it == tables_cache.end()) - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); res = it->second.table; if (it->second.expiration_iterator != cache_expiration_queue.end()) cache_expiration_queue.erase(it->second.expiration_iterator); @@ -230,7 +231,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const { const auto & ast_create = ast->as(); String table_data_path_relative = getTableDataPath(ast_create); - table = createTableFromAST(ast_create, database_name, table_data_path_relative, context_copy, false).second; + table = createTableFromAST(ast_create, getDatabaseName(), table_data_path_relative, context_copy, false).second; } if (!ast || !endsWith(table->getName(), "Log")) @@ -239,7 +240,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const std::lock_guard lock(mutex); auto it = tables_cache.find(table_name); if (it == tables_cache.end()) - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); if (it->second.expiration_iterator != cache_expiration_queue.end()) cache_expiration_queue.erase(it->second.expiration_iterator); @@ -299,6 +300,7 @@ DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, Strings && , iterator(table_names.begin()) , current_storage(nullptr) { + database_name = database.database_name; } void DatabaseLazyIterator::next() diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 2e24b687be5..3d3e5a8aed6 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -42,7 +42,8 @@ public: const String & table_name, IDatabase & to_database, const String & to_table_name, - bool exchange) override; + bool exchange, + bool dictionary) override; void alterTable( const Context & context, diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index cd559172197..221e54ce741 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -58,7 +58,7 @@ void DatabaseMemory::dropTable( ASTPtr DatabaseMemory::getCreateDatabaseQuery() const { auto create_query = std::make_shared(); - create_query->database = database_name; + create_query->database = getDatabaseName(); create_query->set(create_query->storage, std::make_shared()); create_query->storage->set(create_query->storage->engine, makeASTFunction(getEngineName())); return create_query; diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index 56cd5c8088d..3b026bf9468 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -99,7 +99,7 @@ DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, cons if (!remove_or_detach_tables.count(table_name) && (!filter_by_table_name || filter_by_table_name(table_name))) tables[table_name] = modify_time_and_storage.second; - return std::make_unique(tables); + return std::make_unique(tables, database_name); } bool DatabaseMySQL::isTableExist(const String & name, const Context &) const @@ -187,7 +187,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const { const auto & create_query = std::make_shared(); - create_query->database = database_name; + create_query->database = getDatabaseName(); create_query->set(create_query->storage, database_engine_define); return create_query; } @@ -378,11 +378,11 @@ void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & st std::lock_guard lock{mutex}; if (!local_tables_cache.count(table_name)) - throw Exception("Cannot attach table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + + throw Exception("Cannot attach table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " because it does not exist.", ErrorCodes::UNKNOWN_TABLE); if (!remove_or_detach_tables.count(table_name)) - throw Exception("Cannot attach table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + + throw Exception("Cannot attach table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " because it already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); /// We use the new storage to replace the original storage, because the original storage may have been dropped @@ -401,11 +401,11 @@ StoragePtr DatabaseMySQL::detachTable(const String & table_name) std::lock_guard lock{mutex}; if (remove_or_detach_tables.count(table_name)) - throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " is dropped", + throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped", ErrorCodes::TABLE_IS_DROPPED); if (!local_tables_cache.count(table_name)) - throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", + throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); remove_or_detach_tables.emplace(table_name); @@ -441,16 +441,16 @@ void DatabaseMySQL::dropTable(const Context &, const String & table_name, bool / Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); if (remove_or_detach_tables.count(table_name)) - throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " is dropped", + throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped", ErrorCodes::TABLE_IS_DROPPED); if (remove_flag.exists()) - throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(getDatabaseName()) + + throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR); auto table_iter = local_tables_cache.find(table_name); if (table_iter == local_tables_cache.end()) - throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", + throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); remove_or_detach_tables.emplace(table_name); diff --git a/src/Databases/DatabaseMySQL.h b/src/Databases/DatabaseMySQL.h index 70d8dc81e0f..5f45713537c 100644 --- a/src/Databases/DatabaseMySQL.h +++ b/src/Databases/DatabaseMySQL.h @@ -67,7 +67,6 @@ private: ASTPtr database_engine_define; String database_name_in_mysql; - mutable std::mutex mutex; std::atomic quit{false}; std::condition_variable cond; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 799ed041bef..4e829357508 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -20,6 +20,7 @@ #include #include +#include namespace DB @@ -145,7 +146,7 @@ void DatabaseOnDisk::createTable( { const auto & settings = context.getSettingsRef(); const auto & create = query->as(); - assert(getDatabaseName() == create.database && table_name == create.table); + assert(table_name == create.table); /// Create a file with metadata if necessary - if the query is not ATTACH. /// Write the query of `ATTACH table` to it. @@ -216,7 +217,8 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam String table_metadata_path = getObjectMetadataPath(table_name); String table_metadata_path_drop = table_metadata_path + drop_suffix; String table_data_path_relative = getTableDataPath(table_name); - assert(!table_data_path_relative.empty()); + if (table_data_path_relative.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path is empty"); StoragePtr table = detachTable(table_name); bool renamed = false; @@ -248,10 +250,13 @@ void DatabaseOnDisk::renameTable( const String & table_name, IDatabase & to_database, const String & to_table_name, - bool exchange) + bool exchange, + bool dictionary) { if (exchange) throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED); + if (dictionary) + throw Exception("Dictionaries can be renamed only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED); bool from_ordinary_to_atomic = false; bool from_atomic_to_ordinary = false; @@ -305,6 +310,14 @@ void DatabaseOnDisk::renameTable( to_database.createTable(context, to_table_name, table, attach_query); Poco::File(table_metadata_path).remove(); + + /// Special case: usually no actions with symlinks are required when detaching/attaching table, + /// but not when moving from Atomic database to Ordinary + if (from_atomic_to_ordinary) + { + auto & atomic_db = assert_cast(*this); + atomic_db.tryRemoveSymlink(table_name); + } } ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const @@ -332,9 +345,14 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const ASTPtr ast; auto settings = global_context.getSettingsRef(); - auto metadata_dir_path = getMetadataPath(); - auto database_metadata_path = metadata_dir_path.substr(0, metadata_dir_path.size() - 1) + ".sql"; - ast = getCreateQueryFromMetadata(database_metadata_path, true); + { + std::lock_guard lock(mutex); + auto database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"; + ast = parseQueryFromMetadata(log, global_context, database_metadata_path, true); + auto & ast_create_query = ast->as(); + ast_create_query.attach = false; + ast_create_query.database = database_name; + } if (!ast) { /// Handle databases (such as default) for which there are no database.sql files. @@ -462,7 +480,8 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte */ if (remove_empty && query.empty()) { - LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path); + if (logger) + LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path); Poco::File(metadata_file_path).remove(); return nullptr; } @@ -480,12 +499,12 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte return nullptr; auto & create = ast->as(); - if (create.uuid != UUIDHelpers::Nil) + if (!create.table.empty() && create.uuid != UUIDHelpers::Nil) { String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); table_name = unescapeForFileName(table_name); - if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER) + if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger) LOG_WARNING(logger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table); create.table = table_name; } @@ -501,7 +520,7 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metada { auto & ast_create_query = ast->as(); ast_create_query.attach = false; - ast_create_query.database = database_name; + ast_create_query.database = getDatabaseName(); } return ast; diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index d4fb9b2aa17..586491d4d29 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -49,7 +49,8 @@ public: const String & table_name, IDatabase & to_database, const String & to_table_name, - bool exchange) override; + bool exchange, + bool dictionary) override; ASTPtr getCreateDatabaseQuery() const override; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 1e82420298b..45a7c1d2c66 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -129,6 +129,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto if (ast) { auto * create_query = ast->as(); + create_query->database = database_name; std::lock_guard lock{file_names_mutex}; file_names[file_name] = ast; total_dictionaries += create_query->is_dictionary; @@ -165,7 +166,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto context, create_query, *this, - getDatabaseName(), + database_name, getMetadataPath() + name_with_query.first, has_force_restore_data_flag); diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index 9be7e4d8b3e..f724faac637 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -34,21 +34,30 @@ namespace ErrorCodes void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const DictionaryAttachInfo & attach_info) { - String full_name = getDatabaseName() + "." + dictionary_name; + auto dict_id = StorageID(attach_info.create_query); + String internal_name = dict_id.getInternalDictionaryName(); + assert(attach_info.create_query->as().table == dictionary_name); + assert(!dict_id.database_name.empty()); { std::unique_lock lock(mutex); auto [it, inserted] = dictionaries.emplace(dictionary_name, attach_info); if (!inserted) - throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); + throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, + "Dictionary {} already exists.", dict_id.getNameForLogs()); /// Attach the dictionary as table too. try { + /// TODO Make StorageDictionary an owner of IDictionaryBase objects. + /// All DDL operations with dictionaries will work with StorageDictionary table, + /// and StorageDictionary will be responsible for loading of DDL dictionaries. + /// ExternalLoaderDatabaseConfigRepository and other hacks related to ExternalLoader + /// will not be longer required. attachTableUnlocked( dictionary_name, StorageDictionary::create( - StorageID(getDatabaseName(), dictionary_name), - full_name, + dict_id, + internal_name, ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config), StorageDictionary::Location::SameDatabaseAndNameAsDictionary), lock); @@ -60,11 +69,11 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, } } - CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast(ExternalLoaderStatus::NOT_LOADED)); + CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, internal_name, static_cast(ExternalLoaderStatus::NOT_LOADED)); /// We want ExternalLoader::reloadConfig() to find out that the dictionary's config /// has been added and in case `dictionaries_lazy_load == false` to load the dictionary. - reloadDictionaryConfig(full_name); + reloadDictionaryConfig(internal_name); } void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name) @@ -75,20 +84,28 @@ void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name) void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info) { - String full_name = getDatabaseName() + "." + dictionary_name; + auto dict_id = StorageID::createEmpty(); + String internal_name; { std::unique_lock lock(mutex); auto it = dictionaries.find(dictionary_name); if (it == dictionaries.end()) - throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_DICTIONARY); + throw Exception(ErrorCodes::UNKNOWN_DICTIONARY, + "Dictionary {}.{} doesn't exist.", database_name, dictionary_name); + dict_id = StorageID(it->second.create_query); + internal_name = dict_id.getInternalDictionaryName(); + assert(dict_id.table_name == dictionary_name); + assert(!dict_id.database_name.empty()); + attach_info = std::move(it->second); dictionaries.erase(it); /// Detach the dictionary as table too. try { - detachTableUnlocked(dictionary_name, lock); + if (!dict_id.hasUUID()) + detachTableUnlocked(dictionary_name, lock); } catch (...) { @@ -97,11 +114,14 @@ void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_na } } - CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, full_name); + CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, internal_name); /// We want ExternalLoader::reloadConfig() to find out that the dictionary's config /// has been removed and to unload the dictionary. - reloadDictionaryConfig(full_name); + reloadDictionaryConfig(internal_name); + + if (dict_id.hasUUID()) + detachTable(dictionary_name); } void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query) @@ -116,20 +136,22 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S * - rename .sql.tmp to .sql. */ + auto dict_id = StorageID(query); + assert(query->as().table == dictionary_name); + assert(!dict_id.database_name.empty()); + /// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH. /// But there is protection from it - see using DDLGuard in InterpreterCreateQuery. if (isDictionaryExist(dictionary_name)) - throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); + throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {} already exists.", dict_id.getFullTableName()); /// A dictionary with the same full name could be defined in *.xml config files. - String full_name = getDatabaseName() + "." + dictionary_name; - if (external_loader.getCurrentStatus(full_name) != ExternalLoader::Status::NOT_EXIST) - throw Exception( - "Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", - ErrorCodes::DICTIONARY_ALREADY_EXISTS); + if (external_loader.getCurrentStatus(dict_id.getFullNameNotQuoted()) != ExternalLoader::Status::NOT_EXIST) + throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, + "Dictionary {} already exists.", dict_id.getFullNameNotQuoted()); if (isTableExist(dictionary_name, global_context)) - throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", dict_id.getFullTableName()); String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); @@ -162,7 +184,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S { /// load() is called here to force loading the dictionary, wait until the loading is finished, /// and throw an exception if the loading is failed. - external_loader.load(full_name); + external_loader.load(dict_id.getInternalDictionaryName()); } auto config = getDictionaryConfigurationFromAST(query->as()); @@ -178,7 +200,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S /// ExternalDictionariesLoader doesn't know we renamed the metadata path. /// That's why we have to call ExternalLoader::reloadConfig() here. - reloadDictionaryConfig(full_name); + reloadDictionaryConfig(dict_id.getInternalDictionaryName()); /// Everything's ok. succeeded = true; @@ -193,7 +215,8 @@ void DatabaseWithDictionaries::removeDictionary(const Context &, const String & { String dictionary_metadata_path = getObjectMetadataPath(dictionary_name); Poco::File(dictionary_metadata_path).remove(); - CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, getDatabaseName() + "." + dictionary_name); + CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, + StorageID(attach_info.create_query).getInternalDictionaryName()); } catch (...) { @@ -206,14 +229,16 @@ void DatabaseWithDictionaries::removeDictionary(const Context &, const String & DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name) { std::lock_guard lock(mutex); - if (!filter_by_dictionary_name) - return std::make_unique(dictionaries); - - Dictionaries filtered_dictionaries; - for (const auto & dictionary_name : dictionaries | boost::adaptors::map_keys) - if (filter_by_dictionary_name(dictionary_name)) - filtered_dictionaries.emplace_back(dictionary_name); - return std::make_unique(std::move(filtered_dictionaries)); + DictionariesWithID filtered_dictionaries; + for (const auto & dictionary : dictionaries) + { + if (filter_by_dictionary_name && !filter_by_dictionary_name(dictionary.first)) + continue; + filtered_dictionaries.emplace_back(); + filtered_dictionaries.back().first = dictionary.first; + filtered_dictionaries.back().second = dictionary.second.create_query->as().uuid; + } + return std::make_unique(std::move(filtered_dictionaries), database_name); } bool DatabaseWithDictionaries::isDictionaryExist(const String & dictionary_name) const @@ -235,7 +260,7 @@ ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl( ASTPtr ast = it->second.create_query->clone(); auto & create_query = ast->as(); create_query.attach = false; - create_query.database = getDatabaseName(); + create_query.database = database_name; return ast; } } @@ -296,8 +321,11 @@ void DatabaseWithDictionaries::reloadDictionaryConfig(const String & full_name) { /// Ensure that this database is attached to ExternalLoader as a config repository. if (!database_as_config_repo_for_external_loader.load()) - database_as_config_repo_for_external_loader = boost::make_shared( - external_loader.addConfigRepository(std::make_unique(*this))); + { + auto repository = std::make_unique(*this, global_context); + auto remove_repository_callback = external_loader.addConfigRepository(std::move(repository)); + database_as_config_repo_for_external_loader = boost::make_shared(std::move(remove_repository_callback)); + } external_loader.reloadConfig(getDatabaseName(), full_name); } diff --git a/src/Databases/DatabaseWithDictionaries.h b/src/Databases/DatabaseWithDictionaries.h index eb9e105e31d..7f59b5028c4 100644 --- a/src/Databases/DatabaseWithDictionaries.h +++ b/src/Databases/DatabaseWithDictionaries.h @@ -42,12 +42,12 @@ protected: ASTPtr getCreateDictionaryQueryImpl(const String & dictionary_name, bool throw_on_error) const override; std::unordered_map dictionaries; + const ExternalDictionariesLoader & external_loader; private: void detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info); void reloadDictionaryConfig(const String & full_name); - const ExternalDictionariesLoader & external_loader; boost::atomic_shared_ptr database_as_config_repo_for_external_loader; }; diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 47c54fae800..eadfa5f53c6 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes { extern const int TABLE_ALREADY_EXISTS; extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_DATABASE; } DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context) @@ -43,14 +44,14 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Con { std::lock_guard lock(mutex); if (!filter_by_table_name) - return std::make_unique(tables); + return std::make_unique(tables, database_name); Tables filtered_tables; for (const auto & [table_name, storage] : tables) if (filter_by_table_name(table_name)) filtered_tables.emplace(table_name, storage); - return std::make_unique(std::move(filtered_tables)); + return std::make_unique(std::move(filtered_tables), database_name); } bool DatabaseWithOwnTablesBase::empty() const @@ -71,14 +72,15 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n auto it = tables.find(table_name); if (it == tables.end()) - throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", + backQuote(database_name), backQuote(table_name)); res = it->second; tables.erase(it); auto table_id = res->getStorageID(); if (table_id.hasUUID()) { - assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); + assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); } @@ -93,12 +95,17 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock &) { - if (!tables.emplace(table_name, table).second) - throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); auto table_id = table->getStorageID(); + if (table_id.database_name != database_name) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`", + database_name, table_id.database_name); + + if (!tables.emplace(table_name, table).second) + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName()); + if (table_id.hasUUID()) { - assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); + assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic"); DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table); } } @@ -146,7 +153,8 @@ StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name auto it = tables.find(table_name); if (it != tables.end()) return it->second; - throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", + backQuote(database_name), backQuote(table_name)); } } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 4238fd30137..4c7ec1ec637 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -36,7 +36,6 @@ public: ~DatabaseWithOwnTablesBase() override; protected: - mutable std::mutex mutex; Tables tables; Poco::Logger * log; const Context & global_context; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 26b27045be6..34bcb93c5d8 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -23,7 +23,7 @@ struct Settings; struct ConstraintsDescription; struct IndicesDescription; class ASTCreateQuery; -using Dictionaries = std::vector; +using DictionariesWithID = std::vector>; namespace ErrorCodes { @@ -48,6 +48,11 @@ public: virtual ~IDatabaseTablesIterator() = default; virtual UUID uuid() const { return UUIDHelpers::Nil; } + + const String & databaseName() const { assert(!database_name.empty()); return database_name; } + +protected: + String database_name; }; /// Copies list of tables and iterates through such snapshot. @@ -65,12 +70,21 @@ protected: other.it = other.tables.end(); it = tables.begin(); std::advance(it, idx); + database_name = std::move(other.database_name); } public: - DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {} + DatabaseTablesSnapshotIterator(const Tables & tables_, const String & database_name_) + : tables(tables_), it(tables.begin()) + { + database_name = database_name_; + } - DatabaseTablesSnapshotIterator(Tables && tables_) : tables(tables_), it(tables.begin()) {} + DatabaseTablesSnapshotIterator(Tables && tables_, String && database_name_) + : tables(std::move(tables_)), it(tables.begin()) + { + database_name = std::move(database_name_); + } void next() override { ++it; } @@ -85,25 +99,30 @@ public: class DatabaseDictionariesSnapshotIterator { private: - Dictionaries dictionaries; - Dictionaries::iterator it; + DictionariesWithID dictionaries; + DictionariesWithID::iterator it; + String database_name; public: DatabaseDictionariesSnapshotIterator() = default; - DatabaseDictionariesSnapshotIterator(Dictionaries & dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {} - DatabaseDictionariesSnapshotIterator(Dictionaries && dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {} - - DatabaseDictionariesSnapshotIterator(const std::unordered_map & dictionaries_) + DatabaseDictionariesSnapshotIterator(DictionariesWithID & dictionaries_, const String & database_name_) + : dictionaries(dictionaries_), it(dictionaries.begin()), database_name(database_name_) + { + } + DatabaseDictionariesSnapshotIterator(DictionariesWithID && dictionaries_, const String & database_name_) + : dictionaries(dictionaries_), it(dictionaries.begin()), database_name(database_name_) { - boost::range::copy(dictionaries_ | boost::adaptors::map_keys, std::back_inserter(dictionaries)); - it = dictionaries.begin(); } void next() { ++it; } bool isValid() const { return !dictionaries.empty() && it != dictionaries.end(); } - const String & name() const { return *it; } + const String & name() const { return it->first; } + + const UUID & uuid() const { return it->second; } + + const String & databaseName() const { assert(!database_name.empty()); return database_name; } }; using DatabaseTablesIteratorPtr = std::unique_ptr; @@ -228,7 +247,8 @@ public: const String & /*name*/, IDatabase & /*to_database*/, const String & /*to_name*/, - bool /*exchange*/) + bool /*exchange*/, + bool /*dictionary*/) { throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED); } @@ -282,7 +302,19 @@ public: virtual ASTPtr getCreateDatabaseQuery() const = 0; /// Get name of database. - String getDatabaseName() const { return database_name; } + String getDatabaseName() const + { + std::lock_guard lock{mutex}; + return database_name; + } + /// Get UUID of database. + virtual UUID getUUID() const { return UUIDHelpers::Nil; } + + virtual void renameDatabase(const String & /*new_name*/) + { + throw Exception(getEngineName() + ": RENAME DATABASE is not supported", ErrorCodes::NOT_IMPLEMENTED); + } + /// Returns path for persistent data storage if the database supports it, empty string otherwise virtual String getDataPath() const { return {}; } @@ -321,6 +353,7 @@ protected: return nullptr; } + mutable std::mutex mutex; String database_name; }; diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 2f2be695a8f..6fb14251ae5 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -60,8 +60,7 @@ inline size_t CacheDictionary::getCellIdx(const Key id) const CacheDictionary::CacheDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, @@ -72,9 +71,7 @@ CacheDictionary::CacheDictionary( size_t update_queue_push_timeout_milliseconds_, size_t query_wait_timeout_milliseconds_, size_t max_threads_for_updates_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -238,7 +235,7 @@ void CacheDictionary::isInConstantVector(const Key child_id, const PaddedPODArra void CacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const { auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto null_value = StringRef{std::get(attribute.null_values)}; @@ -249,7 +246,7 @@ void CacheDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const { auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); }); } @@ -258,7 +255,7 @@ void CacheDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const { auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; }); } @@ -702,8 +699,7 @@ void registerDictionaryCache(DictionaryFactory & factory) throw Exception{full_name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const size_t strict_max_lifetime_seconds = @@ -712,7 +708,7 @@ void registerDictionaryCache(DictionaryFactory & factory) const size_t max_update_queue_size = config.getUInt64(layout_prefix + ".cache.max_update_queue_size", 100000); if (max_update_queue_size == 0) - throw Exception{name + ": dictionary of layout 'cache' cannot have empty update queue of size 0", + throw Exception{full_name + ": dictionary of layout 'cache' cannot have empty update queue of size 0", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; const bool allow_read_expired_keys = @@ -721,7 +717,7 @@ void registerDictionaryCache(DictionaryFactory & factory) const size_t update_queue_push_timeout_milliseconds = config.getUInt64(layout_prefix + ".cache.update_queue_push_timeout_milliseconds", 10); if (update_queue_push_timeout_milliseconds < 10) - throw Exception{name + ": dictionary of layout 'cache' have too little update_queue_push_timeout", + throw Exception{full_name + ": dictionary of layout 'cache' have too little update_queue_push_timeout", ErrorCodes::BAD_ARGUMENTS}; const size_t query_wait_timeout_milliseconds = @@ -730,12 +726,11 @@ void registerDictionaryCache(DictionaryFactory & factory) const size_t max_threads_for_updates = config.getUInt64(layout_prefix + ".max_threads_for_updates", 4); if (max_threads_for_updates == 0) - throw Exception{name + ": dictionary of layout 'cache' cannot have zero threads for updates.", + throw Exception{full_name + ": dictionary of layout 'cache' cannot have zero threads for updates.", ErrorCodes::BAD_ARGUMENTS}; return std::make_unique( - database, - name, + dict_id, dict_struct, std::move(source_ptr), dict_lifetime, @@ -829,9 +824,9 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr * intended to do a synchronous update. AsyncUpdate thread can touch deallocated memory and explode. * */ update_unit_ptr->can_use_callback = false; - throw DB::Exception( - "Dictionary " + getName() + " source seems unavailable, because " + - toString(timeout_for_wait) + " timeout exceeded.", ErrorCodes::TIMEOUT_EXCEEDED); + throw DB::Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Dictionary {} source seems unavailable, because {} timeout exceeded.", + getDictionaryID().getNameForLogs(), toString(timeout_for_wait)); } @@ -842,10 +837,11 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr void CacheDictionary::tryPushToUpdateQueueOrThrow(UpdateUnitPtr & update_unit_ptr) const { if (!update_queue.tryPush(update_unit_ptr, update_queue_push_timeout_milliseconds)) - throw DB::Exception( - "Cannot push to internal update queue in dictionary " + getFullName() + ". Timelimit of " + - std::to_string(update_queue_push_timeout_milliseconds) + " ms. exceeded. Current queue size is " + - std::to_string(update_queue.size()), ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL); + throw DB::Exception(ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL, + "Cannot push to internal update queue in dictionary {}. " + "Timelimit of {} ms. exceeded. Current queue size is {}", + getDictionaryID().getNameForLogs(), std::to_string(update_queue_push_timeout_milliseconds), + std::to_string(update_queue.size())); } void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const @@ -880,7 +876,8 @@ void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const const auto * id_column = typeid_cast(block.safeGetByPosition(0).column.get()); if (!id_column) - throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH}; + throw Exception{ErrorCodes::TYPE_MISMATCH, + "{}: id column has type different from UInt64.", getDictionaryID().getNameForLogs()}; const auto & ids = id_column->getData(); @@ -945,8 +942,9 @@ void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const last_exception = std::current_exception(); backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count)); - tryLogException(last_exception, log, "Could not update cache dictionary '" + getFullName() + - "', next update is scheduled at " + ext::to_string(backoff_end_time.load())); + tryLogException(last_exception, log, + "Could not update cache dictionary '" + getDictionaryID().getNameForLogs() + + "', next update is scheduled at " + ext::to_string(backoff_end_time.load())); } } diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 7e2fc0441d3..218de240b63 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -50,8 +50,7 @@ class CacheDictionary final : public IDictionary { public: CacheDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, @@ -65,10 +64,6 @@ public: ~CacheDictionary() override; - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "Cache"; } size_t getBytesAllocated() const override { return bytes_allocated + (string_arena ? string_arena->size() : 0); } @@ -89,8 +84,7 @@ public: std::shared_ptr clone() const override { return std::make_shared( - database, - name, + getDictionaryID(), dict_struct, getSourceAndUpdateIfNeeded()->clone(), dict_lifetime, @@ -321,9 +315,6 @@ private: template void isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; /// Dictionary source should be used with mutex diff --git a/src/Dictionaries/CacheDictionary_generate1.cpp b/src/Dictionaries/CacheDictionary_generate1.cpp index edb4b89d550..a041f50ea26 100644 --- a/src/Dictionaries/CacheDictionary_generate1.cpp +++ b/src/Dictionaries/CacheDictionary_generate1.cpp @@ -8,7 +8,7 @@ namespace DB const \ { \ auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ const auto null_value = std::get(attribute.null_values); \ getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return null_value; }); \ } diff --git a/src/Dictionaries/CacheDictionary_generate2.cpp b/src/Dictionaries/CacheDictionary_generate2.cpp index 97fa9e1a365..be28a6302c2 100644 --- a/src/Dictionaries/CacheDictionary_generate2.cpp +++ b/src/Dictionaries/CacheDictionary_generate2.cpp @@ -11,7 +11,7 @@ namespace DB ResultArrayType & out) const \ { \ auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl(attribute, ids, out, [&](const size_t row) { return def[row]; }); \ } diff --git a/src/Dictionaries/CacheDictionary_generate3.cpp b/src/Dictionaries/CacheDictionary_generate3.cpp index 8a94ca6bc20..36195f166db 100644 --- a/src/Dictionaries/CacheDictionary_generate3.cpp +++ b/src/Dictionaries/CacheDictionary_generate3.cpp @@ -8,7 +8,7 @@ namespace DB const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const \ { \ auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl(attribute, ids, out, [&](const size_t) { return def; }); \ } diff --git a/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/src/Dictionaries/ComplexKeyCacheDictionary.cpp index d6a74191a79..0c517699272 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -51,15 +51,12 @@ inline UInt64 ComplexKeyCacheDictionary::getCellIdx(const StringRef key) const ComplexKeyCacheDictionary::ComplexKeyCacheDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, const size_t size_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -80,7 +77,7 @@ void ComplexKeyCacheDictionary::getString( dict_struct.validateKeyTypes(key_types); auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto null_value = StringRef{std::get(attribute.null_values)}; @@ -97,7 +94,7 @@ void ComplexKeyCacheDictionary::getString( dict_struct.validateKeyTypes(key_types); auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsString(attribute, key_columns, out, [&](const size_t row) { return def->getDataAt(row); }); } @@ -112,7 +109,7 @@ void ComplexKeyCacheDictionary::getString( dict_struct.validateKeyTypes(key_types); auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsString(attribute, key_columns, out, [&](const size_t) { return StringRef{def}; }); } @@ -415,10 +412,9 @@ void registerDictionaryComplexKeyCache(DictionaryFactory & factory) throw Exception{full_name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, size); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, size); }; factory.registerLayout("complex_key_cache", create_layout, true); } diff --git a/src/Dictionaries/ComplexKeyCacheDictionary.h b/src/Dictionaries/ComplexKeyCacheDictionary.h index 3d6284c448f..7c9cf6e3c8e 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -42,8 +42,7 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase { public: ComplexKeyCacheDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -51,10 +50,6 @@ public: std::string getKeyDescription() const { return key_description; } - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "ComplexKeyCache"; } size_t getBytesAllocated() const override @@ -78,7 +73,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), dict_lifetime, size); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, size); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -671,9 +666,6 @@ private: bool isEmptyCell(const UInt64 idx) const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp index 6f9761cd064..01d39722d33 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate1.cpp @@ -8,7 +8,7 @@ namespace DB { \ dict_struct.validateKeyTypes(key_types); \ auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ const auto null_value = std::get(attribute.null_values); \ getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return null_value; }); \ } diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp index 297da2c91a0..deb34706f54 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate2.cpp @@ -12,7 +12,7 @@ namespace DB { \ dict_struct.validateKeyTypes(key_types); \ auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); \ } diff --git a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp index 222e9c8ee77..2a84fdc89f6 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp +++ b/src/Dictionaries/ComplexKeyCacheDictionary_generate3.cpp @@ -12,7 +12,7 @@ namespace DB { \ dict_struct.validateKeyTypes(key_types); \ auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl(attribute, key_columns, out, [&](const size_t) { return def; }); \ } diff --git a/src/Dictionaries/ComplexKeyDirectDictionary.cpp b/src/Dictionaries/ComplexKeyDirectDictionary.cpp index 81058f28ff4..814a9f2f504 100644 --- a/src/Dictionaries/ComplexKeyDirectDictionary.cpp +++ b/src/Dictionaries/ComplexKeyDirectDictionary.cpp @@ -15,14 +15,11 @@ namespace ErrorCodes ComplexKeyDirectDictionary::ComplexKeyDirectDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, BlockPtr saved_block_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , saved_block{std::move(saved_block_)} @@ -39,7 +36,7 @@ ComplexKeyDirectDictionary::ComplexKeyDirectDictionary( { \ dict_struct.validateKeyTypes(key_types); \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(attribute.null_values); \ \ @@ -67,7 +64,7 @@ void ComplexKeyDirectDictionary::getString( { dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto & null_value = std::get(attribute.null_values); getItemsStringImpl( @@ -87,7 +84,7 @@ void ComplexKeyDirectDictionary::getString( { \ dict_struct.validateKeyTypes(key_types); \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \ @@ -114,7 +111,7 @@ void ComplexKeyDirectDictionary::getString( dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsStringImpl( attribute, @@ -129,7 +126,7 @@ void ComplexKeyDirectDictionary::getString( { \ dict_struct.validateKeyTypes(key_types); \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -156,7 +153,7 @@ void ComplexKeyDirectDictionary::getString( dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); ComplexKeyDirectDictionary::getItemsStringImpl( attribute, @@ -588,14 +585,13 @@ void registerDictionaryComplexKeyDirect(DictionaryFactory & factory) "for a dictionary of layout 'range_hashed'", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max")) throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS}; - return std::make_unique(database, name, dict_struct, std::move(source_ptr)); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr)); }; factory.registerLayout("complex_key_direct", create_layout, false); } diff --git a/src/Dictionaries/ComplexKeyDirectDictionary.h b/src/Dictionaries/ComplexKeyDirectDictionary.h index e814c5dde82..dc602be103f 100644 --- a/src/Dictionaries/ComplexKeyDirectDictionary.h +++ b/src/Dictionaries/ComplexKeyDirectDictionary.h @@ -25,16 +25,11 @@ class ComplexKeyDirectDictionary final : public IDictionaryBase { public: ComplexKeyDirectDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, BlockPtr saved_block_ = nullptr); - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "ComplexKeyDirect"; } size_t getBytesAllocated() const override { return 0; } @@ -51,7 +46,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -205,9 +200,6 @@ private: template void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/src/Dictionaries/ComplexKeyHashedDictionary.cpp index fc857254df4..676196fabd2 100644 --- a/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -14,16 +14,13 @@ namespace ErrorCodes } ComplexKeyHashedDictionary::ComplexKeyHashedDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, BlockPtr saved_block_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -42,7 +39,7 @@ ComplexKeyHashedDictionary::ComplexKeyHashedDictionary( dict_struct.validateKeyTypes(key_types); \ \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(attribute.null_values); \ \ @@ -74,7 +71,7 @@ void ComplexKeyHashedDictionary::getString( dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto & null_value = StringRef{std::get(attribute.null_values)}; @@ -96,7 +93,7 @@ void ComplexKeyHashedDictionary::getString( dict_struct.validateKeyTypes(key_types); \ \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, \ @@ -130,7 +127,7 @@ void ComplexKeyHashedDictionary::getString( dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -150,7 +147,7 @@ void ComplexKeyHashedDictionary::getString( dict_struct.validateKeyTypes(key_types); \ \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -181,7 +178,7 @@ void ComplexKeyHashedDictionary::getString( dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -753,11 +750,10 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory) if (!dict_struct.key) throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("complex_key_hashed", create_layout, true); } diff --git a/src/Dictionaries/ComplexKeyHashedDictionary.h b/src/Dictionaries/ComplexKeyHashedDictionary.h index 82b2a93b010..baf6628eebd 100644 --- a/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -23,8 +23,7 @@ class ComplexKeyHashedDictionary final : public IDictionaryBase { public: ComplexKeyHashedDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -33,10 +32,6 @@ public: std::string getKeyDescription() const { return key_description; } - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "ComplexKeyHashed"; } size_t getBytesAllocated() const override { return bytes_allocated; } @@ -51,7 +46,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -236,9 +231,6 @@ private: template std::vector getKeys(const Attribute & attribute) const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 5a9fa7979c3..08885bc4d1b 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -15,11 +15,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; -} - enum class AttributeUnderlyingType { utUInt8, @@ -44,15 +39,6 @@ AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type); std::string toString(const AttributeUnderlyingType type); -/// Implicit conversions in dictGet functions is disabled. -inline void checkAttributeType(const std::string & dict_name, const std::string & attribute_name, - AttributeUnderlyingType attribute_type, AttributeUnderlyingType to) -{ - if (attribute_type != to) - throw Exception{dict_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute_type) - + ", expected " + toString(to), ErrorCodes::TYPE_MISMATCH}; -} - /// Min and max lifetimes for a dictionary or it's entry using DictionaryLifetime = ExternalLoadableLifetime; diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 61d8e21341b..1fbfcc07215 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -16,14 +16,11 @@ namespace ErrorCodes DirectDictionary::DirectDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, BlockPtr saved_block_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , saved_block{std::move(saved_block_)} @@ -136,7 +133,7 @@ void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArr void DirectDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(attribute.null_values); \ \ @@ -162,7 +159,7 @@ DECLARE(Decimal128) void DirectDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto & null_value = std::get(attribute.null_values); getItemsStringImpl( @@ -180,7 +177,7 @@ void DirectDictionary::getString(const std::string & attribute_name, const Padde ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \ @@ -205,7 +202,7 @@ void DirectDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsStringImpl( attribute, @@ -219,7 +216,7 @@ void DirectDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -244,7 +241,7 @@ void DirectDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); DirectDictionary::getItemsStringImpl( attribute, @@ -577,14 +574,13 @@ void registerDictionaryDirect(DictionaryFactory & factory) "for a dictionary of layout 'range_hashed'", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max")) throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS}; - return std::make_unique(database, name, dict_struct, std::move(source_ptr)); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr)); }; factory.registerLayout("direct", create_layout, false); } diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 3c0fb375624..18ef5224a8a 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -23,16 +23,11 @@ class DirectDictionary final : public IDictionary { public: DirectDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, BlockPtr saved_block_ = nullptr); - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "Direct"; } size_t getBytesAllocated() const override { return 0; } @@ -47,7 +42,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -206,9 +201,6 @@ private: template void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index b6396d20f3a..47ffdaeb5bd 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -21,16 +21,13 @@ static const auto max_array_size = 500000; FlatDictionary::FlatDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, BlockPtr saved_block_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -110,7 +107,7 @@ void FlatDictionary::isInConstantVector(const Key child_id, const PaddedPODArray void FlatDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(attribute.null_values); \ \ @@ -136,7 +133,7 @@ DECLARE(Decimal128) void FlatDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto & null_value = std::get(attribute.null_values); @@ -155,7 +152,7 @@ void FlatDictionary::getString(const std::string & attribute_name, const PaddedP ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \ @@ -180,7 +177,7 @@ void FlatDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -194,7 +191,7 @@ void FlatDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -219,7 +216,7 @@ void FlatDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); FlatDictionary::getItemsImpl( attribute, @@ -724,11 +721,10 @@ void registerDictionaryFlat(DictionaryFactory & factory) "for a dictionary of layout 'range_hashed'", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("flat", create_layout, false); } diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index 636c7b9d092..2f51c1f5c1b 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -22,18 +22,13 @@ class FlatDictionary final : public IDictionary { public: FlatDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, BlockPtr saved_block_ = nullptr); - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "Flat"; } size_t getBytesAllocated() const override { return bytes_allocated; } @@ -48,7 +43,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -225,9 +220,6 @@ private: PaddedPODArray getIds() const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 62fa52acfe8..85456e8c61c 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -32,17 +32,14 @@ namespace ErrorCodes HashedDictionary::HashedDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, bool sparse_, BlockPtr saved_block_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -133,7 +130,7 @@ void HashedDictionary::isInConstantVector(const Key child_id, const PaddedPODArr const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(attribute.null_values); \ \ @@ -159,7 +156,7 @@ DECLARE(Decimal128) void HashedDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto & null_value = StringRef{std::get(attribute.null_values)}; @@ -178,7 +175,7 @@ void HashedDictionary::getString(const std::string & attribute_name, const Padde ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \ @@ -203,7 +200,7 @@ void HashedDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -217,7 +214,7 @@ void HashedDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const TYPE & def, ResultArrayType & out) const \ { \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -242,7 +239,7 @@ void HashedDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const { const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -788,11 +785,10 @@ void registerDictionaryHashed(DictionaryFactory & factory) "for a dictionary of layout 'range_hashed'", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse); }; using namespace std::placeholders; factory.registerLayout("hashed", diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index a361352a8bd..fd6b93b6d5c 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -26,8 +26,7 @@ class HashedDictionary final : public IDictionary { public: HashedDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -35,10 +34,6 @@ public: bool sparse_, BlockPtr saved_block_ = nullptr); - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return sparse ? "SparseHashed" : "Hashed"; } size_t getBytesAllocated() const override { return bytes_allocated; } @@ -53,7 +48,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -271,9 +266,6 @@ private: template void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 485940bdb80..6bc8d32295a 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -4,19 +4,23 @@ #include #include #include +#include #include #include #include #include "IDictionarySource.h" +#include #include #include +#include namespace DB { namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int TYPE_MISMATCH; } struct IDictionaryBase; @@ -29,22 +33,37 @@ struct IDictionaryBase : public IExternalLoadable { using Key = UInt64; - virtual const std::string & getDatabase() const = 0; - virtual const std::string & getName() const = 0; - virtual const std::string & getFullName() const = 0; + IDictionaryBase(const StorageID & dict_id_) + : dict_id(dict_id_) + , full_name(dict_id.getInternalDictionaryName()) + { + } - const std::string & getLoadableName() const override { return getFullName(); } + const std::string & getFullName() const{ return full_name; } + StorageID getDictionaryID() const + { + std::lock_guard lock{name_mutex}; + return dict_id; + } + + void updateDictionaryName(const StorageID & new_name) const + { + std::lock_guard lock{name_mutex}; + assert(new_name.uuid == dict_id.uuid && dict_id.uuid != UUIDHelpers::Nil); + dict_id = new_name; + } + + const std::string & getLoadableName() const override final { return getFullName(); } /// Specifies that no database is used. /// Sometimes we cannot simply use an empty string for that because an empty string is /// usually replaced with the current database. static constexpr char NO_DATABASE_TAG[] = ""; - std::string_view getDatabaseOrNoDatabaseTag() const + std::string getDatabaseOrNoDatabaseTag() const { - const std::string & database = getDatabase(); - if (!database.empty()) - return database; + if (!dict_id.database_name.empty()) + return dict_id.database_name; return NO_DATABASE_TAG; } @@ -87,11 +106,20 @@ struct IDictionaryBase : public IExternalLoadable { return std::static_pointer_cast(IExternalLoadable::shared_from_this()); } + +private: + mutable std::mutex name_mutex; + mutable StorageID dict_id; + +protected: + const String full_name; }; struct IDictionary : IDictionaryBase { + IDictionary(const StorageID & dict_id_) : IDictionaryBase(dict_id_) {} + virtual bool hasHierarchy() const = 0; virtual void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const = 0; @@ -103,19 +131,22 @@ struct IDictionary : IDictionaryBase virtual void isInVectorVector( const PaddedPODArray & /*child_ids*/, const PaddedPODArray & /*ancestor_ids*/, PaddedPODArray & /*out*/) const { - throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); } virtual void isInVectorConstant(const PaddedPODArray & /*child_ids*/, const Key /*ancestor_id*/, PaddedPODArray & /*out*/) const { - throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); } virtual void isInConstantVector(const Key /*child_id*/, const PaddedPODArray & /*ancestor_ids*/, PaddedPODArray & /*out*/) const { - throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); } void isInConstantConstant(const Key child_id, const Key ancestor_id, UInt8 & out) const @@ -126,4 +157,14 @@ struct IDictionary : IDictionaryBase } }; +/// Implicit conversions in dictGet functions is disabled. +inline void checkAttributeType(const IDictionaryBase * dictionary, const std::string & attribute_name, + AttributeUnderlyingType attribute_type, AttributeUnderlyingType to) +{ + if (attribute_type != to) + throw Exception{ErrorCodes::TYPE_MISMATCH, "{}: type mismatch: attribute {} has type {}, expected {}", + dictionary->getDictionaryID().getNameForLogs(), + attribute_name, toString(attribute_type), toString(to)}; +} + } diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index eb6f0c6387c..04eadbfc0ce 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -20,16 +20,13 @@ namespace ErrorCodes IPolygonDictionary::IPolygonDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, InputType input_type_, PointType point_type_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , dict_lifetime(dict_lifetime_) @@ -40,21 +37,6 @@ IPolygonDictionary::IPolygonDictionary( loadData(); } -const std::string & IPolygonDictionary::getDatabase() const -{ - return database; -} - -const std::string & IPolygonDictionary::getName() const -{ - return name; -} - -const std::string & IPolygonDictionary::getFullName() const -{ - return full_name; -} - std::string IPolygonDictionary::getTypeName() const { return "Polygon"; @@ -186,7 +168,9 @@ void IPolygonDictionary::createAttributes() appendNullValue(attr.underlying_type, attr.null_value); if (attr.hierarchical) - throw Exception{name + ": hierarchical attributes not supported for dictionary of polygonal type", ErrorCodes::TYPE_MISMATCH}; + throw Exception{ErrorCodes::TYPE_MISMATCH, + "{}: hierarchical attributes not supported for dictionary of polygonal type", + getDictionaryID().getNameForLogs()}; } } @@ -297,7 +281,7 @@ size_t IPolygonDictionary::getAttributeIndex(const std::string & attribute_name) const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ResultArrayType & out) const \ { \ const auto ind = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(null_values[ind]); \ \ @@ -327,7 +311,7 @@ void IPolygonDictionary::getString( const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ColumnString * out) const { const auto ind = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString); const auto & null_value = StringRef{std::get(null_values[ind])}; @@ -347,7 +331,7 @@ void IPolygonDictionary::getString( ResultArrayType & out) const \ { \ const auto ind = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ ind, \ @@ -379,7 +363,7 @@ void IPolygonDictionary::getString( ColumnString * const out) const { const auto ind = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString); getItemsImpl( ind, @@ -397,7 +381,7 @@ void IPolygonDictionary::getString( ResultArrayType & out) const \ { \ const auto ind = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ ind, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -426,7 +410,7 @@ void IPolygonDictionary::getString( ColumnString * const out) const { const auto ind = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString); getItemsImpl( ind, diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index ce420463605..75114cff435 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -49,18 +49,13 @@ public: Tuple, }; IPolygonDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, InputType input_type_, PointType point_type_); - const std::string & getDatabase() const override; - const std::string & getName() const override; - const std::string & getFullName() const override; - std::string getTypeName() const override; std::string getKeyDescription() const; @@ -200,9 +195,6 @@ protected: */ std::vector ids; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index 4dd42ac8b6e..6570b112853 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -18,22 +18,20 @@ namespace ErrorCodes } PolygonDictionarySimple::PolygonDictionarySimple( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, InputType input_type_, PointType point_type_): - IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_) + IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_) { } std::shared_ptr PolygonDictionarySimple::clone() const { return std::make_shared( - this->database, - this->name, + this->getDictionaryID(), this->dict_struct, this->source_ptr->clone(), this->dict_lifetime, @@ -57,8 +55,7 @@ bool PolygonDictionarySimple::find(const Point & point, size_t & id) const } PolygonDictionaryIndexEach::PolygonDictionaryIndexEach( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -66,7 +63,7 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach( PointType point_type_, int min_intersections_, int max_depth_) - : IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_), + : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_), grid(min_intersections_, max_depth_, polygons), min_intersections(min_intersections_), max_depth(max_depth_) @@ -83,8 +80,7 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach( std::shared_ptr PolygonDictionaryIndexEach::clone() const { return std::make_shared( - this->database, - this->name, + this->getDictionaryID(), this->dict_struct, this->source_ptr->clone(), this->dict_lifetime, @@ -118,8 +114,7 @@ bool PolygonDictionaryIndexEach::find(const Point & point, size_t & id) const } PolygonDictionaryIndexCell::PolygonDictionaryIndexCell( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -127,7 +122,7 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell( PointType point_type_, size_t min_intersections_, size_t max_depth_) - : IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_), + : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_), index(min_intersections_, max_depth_, polygons), min_intersections(min_intersections_), max_depth(max_depth_) @@ -137,8 +132,7 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell( std::shared_ptr PolygonDictionaryIndexCell::clone() const { return std::make_shared( - this->database, - this->name, + this->getDictionaryID(), this->dict_struct, this->source_ptr->clone(), this->dict_lifetime, @@ -228,6 +222,8 @@ DictionaryPtr createLayout(const std::string & , const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); + if constexpr (std::is_same_v || std::is_same_v) { const auto & layout_prefix = config_prefix + ".layout"; @@ -236,10 +232,10 @@ DictionaryPtr createLayout(const std::string & , const auto & dict_prefix = layout_prefix + "." + keys.front(); size_t max_depth = config.getUInt(dict_prefix + ".max_depth", PolygonDictionary::kMaxDepthDefault); size_t min_intersections = config.getUInt(dict_prefix + ".min_intersections", PolygonDictionary::kMinIntersectionsDefault); - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type, min_intersections, max_depth); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type, min_intersections, max_depth); } else - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type); } void registerDictionaryPolygon(DictionaryFactory & factory) diff --git a/src/Dictionaries/PolygonDictionaryImplementations.h b/src/Dictionaries/PolygonDictionaryImplementations.h index 285569b6829..24910c23430 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.h +++ b/src/Dictionaries/PolygonDictionaryImplementations.h @@ -10,15 +10,14 @@ namespace DB /** Simple implementation of the polygon dictionary. Doesn't generate anything during its construction. * Iterates over all stored polygons for each query, checking each of them in linear time. - * Retrieves the polygon with the smallest area containing the given point. + * Retrieves the polygon with the smallest area containing the given point. * If there is more than one any such polygon may be returned. */ class PolygonDictionarySimple : public IPolygonDictionary { public: PolygonDictionarySimple( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, @@ -32,17 +31,16 @@ private: }; /** A polygon dictionary which generates a recursive grid in order to efficiently cut the number - * of polygons to be checked for a given point. + * of polygons to be checked for a given point. * For more detail see the GridRoot and FinalCell classes. - * Separately, a slab index is built for each individual polygon. This allows to check the - * candidates more efficiently. + * Separately, a slab index is built for each individual polygon. This allows to check the + * candidates more efficiently. */ class PolygonDictionaryIndexEach : public IPolygonDictionary { public: PolygonDictionaryIndexEach( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, @@ -71,8 +69,7 @@ class PolygonDictionaryIndexCell : public IPolygonDictionary { public: PolygonDictionaryIndexCell( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index eed5815333a..eeed581c6f4 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -69,15 +69,12 @@ static bool operator<(const RangeHashedDictionary::Range & left, const RangeHash RangeHashedDictionary::RangeHashedDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -160,8 +157,8 @@ void RangeHashedDictionary::createAttributes() attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); if (attribute.hierarchical) - throw Exception{full_name + ": hierarchical attributes not supported by " + getName() + " dictionary.", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception{ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.", + getDictionaryID().getNameForLogs()}; } } @@ -689,11 +686,10 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) throw Exception{full_name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("range_hashed", create_layout, false); } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index eba10bbbdbb..46ae0390b6a 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -18,17 +18,12 @@ class RangeHashedDictionary final : public IDictionaryBase { public: RangeHashedDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_); - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "RangeHashed"; } size_t getBytesAllocated() const override { return bytes_allocated; } @@ -43,7 +38,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -211,9 +206,6 @@ private: friend struct RangeHashedDIctionaryCallGetBlockInputStreamImpl; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/SSDCacheDictionary.cpp b/src/Dictionaries/SSDCacheDictionary.cpp index 6c789b255d3..1ed9bbf21d0 100644 --- a/src/Dictionaries/SSDCacheDictionary.cpp +++ b/src/Dictionaries/SSDCacheDictionary.cpp @@ -1276,7 +1276,7 @@ void SSDCacheStorage::collectGarbage() } SSDCacheDictionary::SSDCacheDictionary( - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -1287,7 +1287,7 @@ SSDCacheDictionary::SSDCacheDictionary( const size_t read_buffer_size_, const size_t write_buffer_size_, const size_t max_stored_keys_) - : name(name_) + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , dict_lifetime(dict_lifetime_) @@ -1314,7 +1314,7 @@ SSDCacheDictionary::SSDCacheDictionary( const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const \ { \ const auto index = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ const auto null_value = std::get(null_values[index]); /* NOLINT */ \ getItemsNumberImpl(index, ids, out, [&](const size_t) { return null_value; }); /* NOLINT */ \ } @@ -1343,7 +1343,7 @@ SSDCacheDictionary::SSDCacheDictionary( ResultArrayType & out) const \ { \ const auto index = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl( \ index, \ ids, \ @@ -1374,7 +1374,7 @@ SSDCacheDictionary::SSDCacheDictionary( ResultArrayType & out) const \ { \ const auto index = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl( \ index, \ ids, \ @@ -1430,7 +1430,7 @@ void SSDCacheDictionary::getItemsNumberImpl( void SSDCacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const { const auto index = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); const auto null_value = StringRef{std::get(null_values[index])}; @@ -1441,7 +1441,7 @@ void SSDCacheDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const { const auto index = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); getItemsStringImpl(index, ids, out, [&](const size_t row) { return def->getDataAt(row); }); } @@ -1450,7 +1450,7 @@ void SSDCacheDictionary::getString( const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const { const auto index = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); getItemsStringImpl(index, ids, out, [&](const size_t) { return StringRef{def}; }); } @@ -1640,6 +1640,8 @@ void registerDictionarySSDCache(DictionaryFactory & factory) if (dict_struct.key) throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); + if (dict_struct.range_min || dict_struct.range_max) throw Exception{name + ": elements .structure.range_min and .structure.range_max should be defined only " @@ -1686,7 +1688,7 @@ void registerDictionarySSDCache(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; return std::make_unique( - name, dict_struct, std::move(source_ptr), dict_lifetime, path, + dict_id, dict_struct, std::move(source_ptr), dict_lifetime, path, max_partitions_count, file_size / block_size, block_size, read_buffer_size / block_size, write_buffer_size / block_size, max_stored_keys); diff --git a/src/Dictionaries/SSDCacheDictionary.h b/src/Dictionaries/SSDCacheDictionary.h index 5574503c237..7199f0386fb 100644 --- a/src/Dictionaries/SSDCacheDictionary.h +++ b/src/Dictionaries/SSDCacheDictionary.h @@ -300,7 +300,7 @@ class SSDCacheDictionary final : public IDictionary { public: SSDCacheDictionary( - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, @@ -312,10 +312,6 @@ public: size_t write_buffer_size_, size_t max_stored_keys_); - const std::string & getDatabase() const override { return name; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return getName(); } - std::string getTypeName() const override { return "SSDCache"; } size_t getBytesAllocated() const override { return storage.getBytesAllocated(); } @@ -335,8 +331,8 @@ public: std::shared_ptr clone() const override { - return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, path, - max_partitions_count, file_size, block_size, read_buffer_size, write_buffer_size, max_stored_keys); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, + path, max_partitions_count, file_size, block_size, read_buffer_size, write_buffer_size, max_stored_keys); } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp index 21fd4140f70..826a61f7312 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp @@ -1323,7 +1323,7 @@ void SSDComplexKeyCacheStorage::collectGarbage() } SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary( - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -1334,7 +1334,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary( const size_t read_buffer_size_, const size_t write_buffer_size_, const size_t max_stored_keys_) - : name(name_) + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , dict_lifetime(dict_lifetime_) @@ -1364,7 +1364,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary( ResultArrayType & out) const \ { \ const auto index = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ const auto null_value = std::get(null_values[index]); /* NOLINT */ \ getItemsNumberImpl(index, key_columns, key_types, out, [&](const size_t) { return null_value; }); /* NOLINT */ \ } @@ -1394,7 +1394,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary( ResultArrayType & out) const \ { \ const auto index = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl(index, key_columns, key_types, out, [&](const size_t row) { return def[row]; }); /* NOLINT */ \ } DECLARE(UInt8) @@ -1422,7 +1422,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary( ResultArrayType & out) const \ { \ const auto index = getAttributeIndex(attribute_name); \ - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \ getItemsNumberImpl(index, key_columns, key_types, out, [&](const size_t) { return def; }); /* NOLINT */ \ } DECLARE(UInt8) @@ -1488,7 +1488,7 @@ void SSDComplexKeyCacheDictionary::getString( const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const { const auto index = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); const auto null_value = StringRef{std::get(null_values[index])}; @@ -1501,7 +1501,7 @@ void SSDComplexKeyCacheDictionary::getString( const ColumnString * const def, ColumnString * const out) const { const auto index = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); getItemsStringImpl(index, key_columns, key_types, out, [&](const size_t row) { return def->getDataAt(row); }); } @@ -1514,7 +1514,7 @@ void SSDComplexKeyCacheDictionary::getString( ColumnString * const out) const { const auto index = getAttributeIndex(attribute_name); - checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString); getItemsStringImpl(index, key_columns, key_types, out, [&](const size_t) { return StringRef{def}; }); } @@ -1736,6 +1736,8 @@ void registerDictionarySSDComplexKeyCache(DictionaryFactory & factory) const std::string & config_prefix, DictionarySourcePtr source_ptr) -> DictionaryPtr { + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); + if (dict_struct.id) throw Exception{"'id' is not supported for dictionary of layout 'complex_key_cache'", ErrorCodes::UNSUPPORTED_METHOD}; @@ -1785,7 +1787,7 @@ void registerDictionarySSDComplexKeyCache(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; return std::make_unique( - name, dict_struct, std::move(source_ptr), dict_lifetime, path, + dict_id, dict_struct, std::move(source_ptr), dict_lifetime, path, max_partitions_count, file_size / block_size, block_size, read_buffer_size / block_size, write_buffer_size / block_size, max_stored_keys); diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.h b/src/Dictionaries/SSDComplexKeyCacheDictionary.h index b9a8466d8c9..89e88982eee 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.h +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.h @@ -522,7 +522,7 @@ class SSDComplexKeyCacheDictionary final : public IDictionaryBase { public: SSDComplexKeyCacheDictionary( - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -534,10 +534,6 @@ public: const size_t write_buffer_size_, const size_t max_stored_keys_); - const std::string & getDatabase() const override { return name; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return getName(); } - std::string getKeyDescription() const { return dict_struct.getKeyDescription(); } std::string getTypeName() const override { return "SSDComplexKeyCache"; } @@ -559,7 +555,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, path, + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, path, max_partitions_count, file_size, block_size, read_buffer_size, write_buffer_size, max_stored_keys); } diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index 4da52b1cd5a..d8267047b92 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -47,15 +47,12 @@ static void validateKeyTypes(const DataTypes & key_types) TrieDictionary::TrieDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_) - : database(database_) - , name(name_) - , full_name{database_.empty() ? name_ : (database_ + "." + name_)} + : IDictionaryBase(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -80,7 +77,7 @@ TrieDictionary::~TrieDictionary() validateKeyTypes(key_types); \ \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ const auto null_value = std::get(attribute.null_values); \ \ @@ -112,7 +109,7 @@ void TrieDictionary::getString( validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); const auto & null_value = StringRef{std::get(attribute.null_values)}; @@ -134,7 +131,7 @@ void TrieDictionary::getString( validateKeyTypes(key_types); \ \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, \ @@ -168,7 +165,7 @@ void TrieDictionary::getString( validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -188,7 +185,7 @@ void TrieDictionary::getString( validateKeyTypes(key_types); \ \ const auto & attribute = getAttribute(attribute_name); \ - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \ \ getItemsImpl( \ attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ @@ -219,7 +216,7 @@ void TrieDictionary::getString( validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); - checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString); getItemsImpl( attribute, @@ -770,12 +767,11 @@ void registerDictionaryTrie(DictionaryFactory & factory) if (!dict_struct.key) throw Exception{"'key' is required for dictionary of layout 'ip_trie'", ErrorCodes::BAD_ARGUMENTS}; - const String database = config.getString(config_prefix + ".database", ""); - const String name = config.getString(config_prefix + ".name"); + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); // This is specialised trie for storing IPv4 and IPv6 prefixes. - return std::make_unique(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("ip_trie", create_layout, true); } diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 063a73b0346..1849f161935 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -23,8 +23,7 @@ class TrieDictionary final : public IDictionaryBase { public: TrieDictionary( - const std::string & database_, - const std::string & name_, + const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, @@ -34,10 +33,6 @@ public: std::string getKeyDescription() const { return key_description; } - const std::string & getDatabase() const override { return database; } - const std::string & getName() const override { return name; } - const std::string & getFullName() const override { return full_name; } - std::string getTypeName() const override { return "Trie"; } size_t getBytesAllocated() const override { return bytes_allocated; } @@ -52,7 +47,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -230,9 +225,6 @@ private: Columns getKeyColumns() const; - const std::string database; - const std::string name; - const std::string full_name; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 30e0ac8bdc7..c194123f047 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -469,6 +469,14 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer AutoPtr database(xml_document->createTextNode(!database_.empty() ? database_ : query.database)); database_element->appendChild(database); + if (query.uuid != UUIDHelpers::Nil) + { + AutoPtr uuid_element(xml_document->createElement("uuid")); + current_dictionary->appendChild(uuid_element); + AutoPtr uuid(xml_document->createTextNode(toString(query.uuid))); + uuid_element->appendChild(uuid); + } + AutoPtr structure_element(xml_document->createElement("structure")); current_dictionary->appendChild(structure_element); Names pk_attrs = getPrimaryKeyColumns(query.dictionary->primary_key); diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index e2cdc6d5e57..609c247ce42 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -85,8 +85,9 @@ public: auto dict = std::atomic_load(&dictionary); if (dict) return dict; - dict = external_loader.getDictionary(dictionary_name); - context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getName()); + String resolved_name = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name); + dict = external_loader.getDictionary(resolved_name); + context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName()); std::atomic_store(&dictionary, dict); return dict; } diff --git a/src/Functions/h3ToChildren.cpp b/src/Functions/h3ToChildren.cpp index 6c1b685bcd9..d9e402231f7 100644 --- a/src/Functions/h3ToChildren.cpp +++ b/src/Functions/h3ToChildren.cpp @@ -5,17 +5,26 @@ #include #include #include +#include #include +#include #include +static constexpr size_t MAX_ARRAY_SIZE = 1 << 30; + + namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int TOO_LARGE_ARRAY_SIZE; } + class FunctionH3ToChildren : public IFunction { public: @@ -63,7 +72,16 @@ public: const UInt64 parent_hindex = col_hindex->getUInt(row); const UInt8 child_resolution = col_resolution->getUInt(row); - const auto vec_size = maxH3ToChildrenSize(parent_hindex, child_resolution); + if (child_resolution > MAX_H3_RES) + throw Exception("The argument 'resolution' (" + toString(child_resolution) + ") of function " + getName() + + " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + const size_t vec_size = maxH3ToChildrenSize(parent_hindex, child_resolution); + if (vec_size > MAX_ARRAY_SIZE) + throw Exception("The result of function" + getName() + + " (array of " + toString(vec_size) + " elements) will be too large with resolution argument = " + + toString(child_resolution), ErrorCodes::TOO_LARGE_ARRAY_SIZE); + hindex_vec.resize(vec_size); h3ToChildren(parent_hindex, child_resolution, hindex_vec.data()); diff --git a/src/Functions/h3ToParent.cpp b/src/Functions/h3ToParent.cpp index 981628ae8f8..2f6a9f3264d 100644 --- a/src/Functions/h3ToParent.cpp +++ b/src/Functions/h3ToParent.cpp @@ -3,17 +3,22 @@ #include #include #include +#include #include +#include #include namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; } + class FunctionH3ToParent : public IFunction { public: @@ -57,6 +62,10 @@ public: const UInt64 hindex = col_hindex->getUInt(row); const UInt8 resolution = col_resolution->getUInt(row); + if (resolution > MAX_H3_RES) + throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName() + + " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + UInt64 res = h3ToParent(hindex, resolution); dst_data[row] = res; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 2a2c66341dc..9a0ee7fed86 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -521,6 +521,39 @@ void NO_INLINE Aggregator::executeImplBatch( size_t rows, AggregateFunctionInstruction * aggregate_instructions) const { + /// Optimization for special case when there are no aggregate functions. + if (params.aggregates_size == 0) + { + /// For all rows. + AggregateDataPtr place = aggregates_pool->alloc(0); + for (size_t i = 0; i < rows; ++i) + state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place); + return; + } + + /// Optimization for special case when aggregating by 8bit key. + if constexpr (std::is_same_v) + { + for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) + { + inst->batch_that->addBatchLookupTable8( + rows, + reinterpret_cast(method.data.data()), + inst->state_offset, + [&](AggregateDataPtr & aggregate_data) + { + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + }, + state.getKeyData(), + inst->batch_arguments, + aggregates_pool); + } + return; + } + + /// Generic case. + PODArray places(rows); /// For all rows. diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 44024b20061..24f3814dc80 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -68,7 +68,7 @@ class IBlockOutputStream; using AggregatedDataWithoutKey = AggregateDataPtr; -using AggregatedDataWithUInt8Key = FixedImplicitZeroHashMap; +using AggregatedDataWithUInt8Key = FixedImplicitZeroHashMapWithCalculatedSize; using AggregatedDataWithUInt16Key = FixedImplicitZeroHashMap; using AggregatedDataWithUInt32Key = HashMap>; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 5c41b4a1fc0..93573b32bbd 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -285,7 +285,7 @@ void SelectStreamFactory::createForShard( } }; - res.emplace_back(createDelayedPipe(header, lazily_create_stream)); + res.emplace_back(createDelayedPipe(header, lazily_create_stream, add_totals, add_extremes)); } else emplace_remote_stream(); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 49b79ad0314..28a21dd0929 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include @@ -144,9 +145,12 @@ void DatabaseCatalog::shutdownImpl() for (auto & database : current_databases) database.second->shutdown(); + tables_marked_dropped.clear(); + std::lock_guard lock(databases_mutex); assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end()); databases.clear(); + db_uuid_map.clear(); view_dependencies.clear(); } @@ -215,6 +219,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( auto table = database->tryGetTable(table_id.table_name, context); if (!table && exception) exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); + if (!table) + database = nullptr; return {database, table}; } @@ -250,7 +256,11 @@ void DatabaseCatalog::attachDatabase(const String & database_name, const Databas { std::lock_guard lock{databases_mutex}; assertDatabaseDoesntExistUnlocked(database_name); - databases[database_name] = database; + databases.emplace(database_name, database); + UUID db_uuid = database->getUUID(); + assert((db_uuid != UUIDHelpers::Nil) ^ (dynamic_cast(database.get()) == nullptr)); + if (db_uuid != UUIDHelpers::Nil) + db_uuid_map.emplace(db_uuid, database); } @@ -259,13 +269,18 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d if (database_name == TEMPORARY_DATABASE) throw Exception("Cannot detach database with temporary tables.", ErrorCodes::DATABASE_ACCESS_DENIED); - std::shared_ptr db; + DatabasePtr db; { std::lock_guard lock{databases_mutex}; assertDatabaseExistsUnlocked(database_name); db = databases.find(database_name)->second; + db_uuid_map.erase(db->getUUID()); + databases.erase(database_name); + } - if (check_empty) + if (check_empty) + { + try { if (!db->empty()) throw Exception("New table appeared in database being dropped or detached. Try again.", @@ -274,8 +289,11 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d if (!drop && database_atomic) database_atomic->assertCanBeDetached(false); } - - databases.erase(database_name); + catch (...) + { + attachDatabase(database_name, db); + throw; + } } db->shutdown(); @@ -295,6 +313,17 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d return db; } +void DatabaseCatalog::updateDatabaseName(const String & old_name, const String & new_name) +{ + std::lock_guard lock{databases_mutex}; + assert(databases.find(new_name) == databases.end()); + auto it = databases.find(old_name); + assert(it != databases.end()); + auto db = it->second; + databases.erase(it); + databases.emplace(new_name, db); +} + DatabasePtr DatabaseCatalog::getDatabase(const String & database_name) const { std::lock_guard lock{databases_mutex}; @@ -312,6 +341,25 @@ DatabasePtr DatabaseCatalog::tryGetDatabase(const String & database_name) const return it->second; } +DatabasePtr DatabaseCatalog::getDatabase(const UUID & uuid) const +{ + std::lock_guard lock{databases_mutex}; + auto it = db_uuid_map.find(uuid); + if (it == db_uuid_map.end()) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database UUID {} does not exist", toString(uuid)); + return it->second; +} + +DatabasePtr DatabaseCatalog::tryGetDatabase(const UUID & uuid) const +{ + assert(uuid != UUIDHelpers::Nil); + std::lock_guard lock{databases_mutex}; + auto it = db_uuid_map.find(uuid); + if (it == db_uuid_map.end()) + return {}; + return it->second; +} + bool DatabaseCatalog::isDatabaseExist(const String & database_name) const { assert(!database_name.empty()); @@ -717,6 +765,31 @@ String DatabaseCatalog::getPathForUUID(const UUID & uuid) return toString(uuid).substr(0, uuid_prefix_len) + '/' + toString(uuid) + '/'; } +String DatabaseCatalog::resolveDictionaryName(const String & name) const +{ + /// If it's dictionary from Atomic database, then we need to convert qualified name to UUID. + /// Try to split name and get id from associated StorageDictionary. + /// If something went wrong, return name as is. + + /// TODO support dot in name for dictionaries in Atomic databases + auto pos = name.find('.'); + if (pos == std::string::npos || name.find('.', pos + 1) != std::string::npos) + return name; + String maybe_database_name = name.substr(0, pos); + String maybe_table_name = name.substr(pos + 1); + + auto db_and_table = tryGetDatabaseAndTable({maybe_database_name, maybe_table_name}, *global_context); + if (!db_and_table.first) + return name; + assert(db_and_table.second); + if (db_and_table.first->getUUID() == UUIDHelpers::Nil) + return name; + if (db_and_table.second->getName() != "Dictionary") + return name; + + return toString(db_and_table.second->getStorageID().uuid); +} + DDLGuard::DDLGuard(Map & map_, std::unique_lock guards_lock_, const String & elem) : map(map_), guards_lock(std::move(guards_lock_)) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 540568927cc..ad90fcc30f2 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -123,10 +123,13 @@ public: void attachDatabase(const String & database_name, const DatabasePtr & database); DatabasePtr detachDatabase(const String & database_name, bool drop = false, bool check_empty = true); + void updateDatabaseName(const String & old_name, const String & new_name); /// database_name must be not empty DatabasePtr getDatabase(const String & database_name) const; DatabasePtr tryGetDatabase(const String & database_name) const; + DatabasePtr getDatabase(const UUID & uuid) const; + DatabasePtr tryGetDatabase(const UUID & uuid) const; bool isDatabaseExist(const String & database_name) const; Databases getDatabases() const; @@ -168,6 +171,9 @@ public: String getPathForDroppedMetadata(const StorageID & table_id) const; void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false); + /// Try convert qualified dictionary name to persistent UUID + String resolveDictionaryName(const String & name) const; + private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -211,6 +217,8 @@ private: static constexpr size_t reschedule_time_ms = 100; private: + using UUIDToDatabaseMap = std::unordered_map; + /// For some reason Context is required to get Storage from Database object Context * global_context; mutable std::mutex databases_mutex; @@ -218,6 +226,7 @@ private: ViewDependencies view_dependencies; Databases databases; + UUIDToDatabaseMap db_uuid_map; UUIDToStorageMap uuid_map; Poco::Logger * log; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index ebbac4c5471..c735dd76911 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -18,7 +18,7 @@ ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_) : ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader")) , context(context_) { - setConfigSettings({"dictionary", "name", "database"}); + setConfigSettings({"dictionary", "name", "database", "uuid"}); enableAsyncLoading(true); enablePeriodicUpdates(true); } diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index cd46845e2ed..e8df205760a 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -290,18 +290,27 @@ private: continue; } - String object_name = file_contents.getString(key + "." + settings.external_name); + /// Use uuid as name if possible + String object_uuid = file_contents.getString(key + "." + settings.external_uuid, ""); + String object_name; + if (object_uuid.empty()) + object_name = file_contents.getString(key + "." + settings.external_name); + else + object_name = object_uuid; if (object_name.empty()) { LOG_WARNING(log, "{}: node '{}' defines {} with an empty name. It's not allowed", path, key, type_name); continue; } - String database; - if (!settings.external_database.empty()) - database = file_contents.getString(key + "." + settings.external_database, ""); - if (!database.empty()) - object_name = database + "." + object_name; + if (object_uuid.empty()) + { + String database; + if (!settings.external_database.empty()) + database = file_contents.getString(key + "." + settings.external_database, ""); + if (!database.empty()) + object_name = database + "." + object_name; + } objects.emplace(object_name, key); } diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 542a40e6cb2..57b711d7f21 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -27,6 +27,7 @@ struct ExternalLoaderConfigSettings std::string external_config; std::string external_name; std::string external_database; + std::string external_uuid; }; /** Interface for manage user-defined objects. diff --git a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp index 5f8f6f7c431..bd29bfb8970 100644 --- a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp @@ -1,5 +1,9 @@ #include +#include +#include #include +#include +#include namespace DB @@ -12,11 +16,25 @@ namespace ErrorCodes namespace { - String trimDatabaseName(const std::string & loadable_definition_name, const String & database_name) + String trimDatabaseName(const std::string & loadable_definition_name, const String & database_name, + const IDatabase & database, const Context & global_context) { + bool is_atomic_database = database.getUUID() != UUIDHelpers::Nil; + if (is_atomic_database) + { + /// We do not know actual database and dictionary names here + auto dict_id = StorageID::createEmpty(); + dict_id.uuid = parseFromString(loadable_definition_name); + assert(dict_id.uuid != UUIDHelpers::Nil); + /// Get associated StorageDictionary by UUID + auto table = DatabaseCatalog::instance().getTable(dict_id, global_context); + auto dict_id_with_names = table->getStorageID(); + return dict_id_with_names.table_name; + } + if (!startsWith(loadable_definition_name, database_name)) - throw Exception( - "Loadable '" + loadable_definition_name + "' is not from database '" + database_name, ErrorCodes::UNKNOWN_DICTIONARY); + throw Exception(ErrorCodes::UNKNOWN_DICTIONARY, + "Loadable '{}' is not from database '{}'", loadable_definition_name, database_name); /// dbname.loadable_name ///--> remove <--- return loadable_definition_name.substr(database_name.length() + 1); @@ -24,34 +42,45 @@ namespace } -ExternalLoaderDatabaseConfigRepository::ExternalLoaderDatabaseConfigRepository(IDatabase & database_) - : database_name(database_.getDatabaseName()) +ExternalLoaderDatabaseConfigRepository::ExternalLoaderDatabaseConfigRepository(IDatabase & database_, const Context & global_context_) + : global_context(global_context_.getGlobalContext()) + , database_name(database_.getDatabaseName()) , database(database_) { } LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std::string & loadable_definition_name) { - return database.getDictionaryConfiguration(trimDatabaseName(loadable_definition_name, database_name)); + auto dict_name = trimDatabaseName(loadable_definition_name, database_name, database, global_context); + return database.getDictionaryConfiguration(dict_name); } bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name) { - return database.isDictionaryExist(trimDatabaseName(loadable_definition_name, database_name)); + auto dict_name = trimDatabaseName(loadable_definition_name, database_name, database, global_context); + return database.isDictionaryExist(dict_name); } Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name) { - return database.getObjectMetadataModificationTime(trimDatabaseName(loadable_definition_name, database_name)); + auto dict_name = trimDatabaseName(loadable_definition_name, database_name, database, global_context); + return database.getObjectMetadataModificationTime(dict_name); } std::set ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames() { std::set result; auto itr = database.getDictionariesIterator(); + bool is_atomic_database = database.getUUID() != UUIDHelpers::Nil; while (itr && itr->isValid()) { - result.insert(database_name + "." + itr->name()); + if (is_atomic_database) + { + assert(itr->uuid() != UUIDHelpers::Nil); + result.insert(toString(itr->uuid())); + } + else + result.insert(database_name + "." + itr->name()); itr->next(); } return result; diff --git a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h index d800db7a8e1..79bd0009b03 100644 --- a/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h +++ b/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h @@ -12,7 +12,7 @@ namespace DB class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository { public: - ExternalLoaderDatabaseConfigRepository(IDatabase & database_); + ExternalLoaderDatabaseConfigRepository(IDatabase & database_, const Context & global_context_); const std::string & getName() const override { return database_name; } @@ -25,6 +25,7 @@ public: LoadablesConfigurationPtr load(const std::string & loadable_definition_name) override; private: + const Context & global_context; const String database_name; IDatabase & database; }; diff --git a/src/Interpreters/ExternalModelsLoader.cpp b/src/Interpreters/ExternalModelsLoader.cpp index 0300bd44949..4e9ddb78241 100644 --- a/src/Interpreters/ExternalModelsLoader.cpp +++ b/src/Interpreters/ExternalModelsLoader.cpp @@ -14,7 +14,7 @@ ExternalModelsLoader::ExternalModelsLoader(Context & context_) : ExternalLoader("external model", &Poco::Logger::get("ExternalModelsLoader")) , context(context_) { - setConfigSettings({"model", "name", {}}); + setConfigSettings({"model", "name", {}, {}}); enablePeriodicUpdates(true); } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 27294a57675..a807a9fa4ee 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -33,7 +33,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_TYPE_OF_FIELD; extern const int NOT_IMPLEMENTED; extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int INCOMPATIBLE_TYPE_OF_JOIN; @@ -158,25 +157,21 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s } else if (strictness == ASTTableJoin::Strictness::Asof) { - if (kind != ASTTableJoin::Kind::Left and kind != ASTTableJoin::Kind::Inner) - throw Exception("ASOF only supports LEFT and INNER as base joins", ErrorCodes::NOT_IMPLEMENTED); + /// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages. + /// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'. + if (!isLeft(kind) && !isInner(kind)) + throw Exception("Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported", ErrorCodes::NOT_IMPLEMENTED); + + if (key_columns.size() <= 1) + throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR); + + if (right_table_keys.getByName(key_names_right.back()).type->isNullable()) + throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); - const IColumn * asof_column = key_columns.back(); size_t asof_size; - - asof_type = AsofRowRefs::getTypeSize(asof_column, asof_size); - if (!asof_type) - { - std::string msg = "ASOF join not supported for type: "; - msg += asof_column->getFamilyName(); - throw Exception(msg, ErrorCodes::BAD_TYPE_OF_FIELD); - } - + asof_type = AsofRowRefs::getTypeSize(*key_columns.back(), asof_size); key_columns.pop_back(); - if (key_columns.empty()) - throw Exception("ASOF join cannot be done without a joining column", ErrorCodes::SYNTAX_ERROR); - /// this is going to set up the appropriate hash table for the direct lookup part of the join /// However, this does not depend on the size of the asof join key (as that goes into the BST) /// Therefore, add it back in such that it can be extracted appropriately from the full stored @@ -248,11 +243,6 @@ HashJoin::Type HashJoin::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & return Type::hashed; } -static const IColumn * extractAsofColumn(const ColumnRawPtrs & key_columns) -{ - return key_columns.back(); -} - template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) { @@ -428,14 +418,15 @@ namespace } static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, - const IColumn * asof_column) + const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); typename Map::mapped_type * time_series_map = &emplace_result.getMapped(); + TypeIndex asof_type = *join.getAsofType(); if (emplace_result.isInserted()) - time_series_map = new (time_series_map) typename Map::mapped_type(join.getAsofType()); - time_series_map->insert(join.getAsofType(), asof_column, stored_block, i); + time_series_map = new (time_series_map) typename Map::mapped_type(asof_type); + time_series_map->insert(asof_type, asof_column, stored_block, i); } }; @@ -451,7 +442,7 @@ namespace const IColumn * asof_column [[maybe_unused]] = nullptr; if constexpr (is_asof_join) - asof_column = extractAsofColumn(key_columns); + asof_column = key_columns.back(); auto key_getter = createKeyGetter(key_columns, key_sizes); @@ -461,7 +452,7 @@ namespace continue; if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, asof_column); + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else @@ -614,21 +605,22 @@ class AddedColumns public: using TypeAndNames = std::vector>; - AddedColumns(const Block & sample_block_with_columns_to_add, - const Block & block_with_columns_to_add, + AddedColumns(const Block & block_with_columns_to_add, const Block & block, const Block & saved_block_sample, - const ColumnsWithTypeAndName & extras, - const HashJoin & join_, + const HashJoin & join, const ColumnRawPtrs & key_columns_, - const Sizes & key_sizes_) - : join(join_) - , key_columns(key_columns_) + const Sizes & key_sizes_, + bool is_asof_join) + : key_columns(key_columns_) , key_sizes(key_sizes_) , rows_to_add(block.rows()) - , need_filter(false) + , asof_type(join.getAsofType()) + , asof_inequality(join.getAsofInequality()) { - size_t num_columns_to_add = sample_block_with_columns_to_add.columns(); + size_t num_columns_to_add = block_with_columns_to_add.columns(); + if (is_asof_join) + ++num_columns_to_add; columns.reserve(num_columns_to_add); type_name.reserve(num_columns_to_add); @@ -641,8 +633,12 @@ public: addColumn(src_column); } - for (const auto & extra : extras) - addColumn(extra); + if (is_asof_join) + { + const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn(); + addColumn(right_asof_column); + left_asof_key = key_columns.back(); + } for (auto & tn : type_name) right_indexes.push_back(saved_block_sample.getPositionByName(tn.second)); @@ -680,18 +676,25 @@ public: } } - const HashJoin & join; + TypeIndex asofType() const { return *asof_type; } + ASOF::Inequality asofInequality() const { return asof_inequality; } + const IColumn & leftAsofKey() const { return *left_asof_key; } + const ColumnRawPtrs & key_columns; const Sizes & key_sizes; size_t rows_to_add; std::unique_ptr offsets_to_replicate; - bool need_filter; + bool need_filter = false; private: TypeAndNames type_name; MutableColumns columns; std::vector right_indexes; size_t lazy_defaults_count = 0; + /// for ASOF + std::optional asof_type; + ASOF::Inequality asof_inequality; + const IColumn * left_asof_key = nullptr; void addColumn(const ColumnWithTypeAndName & src_column) { @@ -760,10 +763,6 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added if constexpr (need_replication) added_columns.offsets_to_replicate = std::make_unique(rows); - const IColumn * asof_column [[maybe_unused]] = nullptr; - if constexpr (is_asof_join) - asof_column = extractAsofColumn(added_columns.key_columns); - auto key_getter = createKeyGetter(added_columns.key_columns, added_columns.key_sizes); IColumn::Offset current_offset = 0; @@ -790,8 +789,11 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added if constexpr (is_asof_join) { - const HashJoin & join = added_columns.join; - if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i)) + TypeIndex asof_type = added_columns.asofType(); + ASOF::Inequality asof_inequality = added_columns.asofInequality(); + const IColumn & left_asof_key = added_columns.leftAsofKey(); + + if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i)) { setUsed(filter, i); mapped.setUsed(); @@ -932,11 +934,11 @@ void HashJoin::joinBlockImpl( /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. Columns materialized_keys = JoinCommon::materializeColumns(block, key_names_left); - ColumnRawPtrs key_columns = JoinCommon::getRawPointers(materialized_keys); + ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys); /// Keys with NULL value in any column won't join to anything. ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(left_key_columns, null_map); size_t existing_columns = block.columns(); @@ -957,12 +959,8 @@ void HashJoin::joinBlockImpl( * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. * For ASOF, the last column is used as the ASOF column */ - ColumnsWithTypeAndName extras; - if constexpr (is_asof_join) - extras.push_back(right_table_keys.getByName(key_names_right.back())); - AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, savedBlockSample(), - extras, *this, key_columns, key_sizes); + AddedColumns added_columns(block_with_columns_to_add, block, savedBlockSample(), *this, left_key_columns, key_sizes, is_asof_join); bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = need_filter || has_required_right_keys; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 67d83d27a6d..fb879e2c507 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -191,10 +191,16 @@ public: ASTTableJoin::Kind getKind() const { return kind; } ASTTableJoin::Strictness getStrictness() const { return strictness; } - TypeIndex getAsofType() const { return *asof_type; } + const std::optional & getAsofType() const { return asof_type; } ASOF::Inequality getAsofInequality() const { return asof_inequality; } bool anyTakeLastRow() const { return any_take_last_row; } + const ColumnWithTypeAndName & rightAsofKeyColumn() const + { + /// It should be nullable if nullable_right_side is true + return savedBlockSample().getByName(key_names_right.back()); + } + /// Different types of keys for maps. #define APPLY_FOR_JOIN_VARIANTS(M) \ M(key8) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ebd6c1e8de4..cd2e45323d1 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include @@ -47,6 +47,7 @@ #include #include +#include #include @@ -76,6 +77,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } +namespace fs = std::filesystem; InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) @@ -98,7 +100,27 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); } - if (!create.storage) + + /// Will write file with database metadata, if needed. + String database_name_escaped = escapeForFileName(database_name); + fs::path metadata_path = fs::canonical(context.getPath()); + fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); + fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); + + if (!create.storage && create.attach) + { + if (!fs::exists(metadata_file_path)) + throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE); + /// Short syntax: try read database definition from file + auto ast = DatabaseOnDisk::parseQueryFromMetadata(nullptr, context, metadata_file_path); + create = ast->as(); + if (!create.table.empty() || !create.storage) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Metadata file {} contains incorrect CREATE DATABASE query", metadata_file_path); + create.attach = true; + create.attach_short_syntax = true; + create.database = database_name; + } + else if (!create.storage) { /// For new-style databases engine is explicitly specified in .sql /// When attaching old-style database during server startup, we must always use Ordinary engine @@ -119,20 +141,39 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "Atomic" && !context.getSettingsRef().allow_experimental_database_atomic && !internal) - throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.", - ErrorCodes::UNKNOWN_DATABASE_ENGINE); + if (create.storage->engine->name == "Atomic") + { + if (!context.getSettingsRef().allow_experimental_database_atomic && !internal) + throw Exception("Atomic is an experimental database engine. " + "Enable allow_experimental_database_atomic to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); - String database_name_escaped = escapeForFileName(database_name); - String path = context.getPath(); - String metadata_path = path + "metadata/" + database_name_escaped + "/"; - DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, context); + if (create.attach && create.uuid == UUIDHelpers::Nil) + throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY); + else if (create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); - /// Will write file with database metadata, if needed. - String metadata_file_tmp_path = path + "metadata/" + database_name_escaped + ".sql.tmp"; - String metadata_file_path = path + "metadata/" + database_name_escaped + ".sql"; + metadata_path = metadata_path / "store" / DatabaseCatalog::getPathForUUID(create.uuid); - bool need_write_metadata = !create.attach; + if (!create.attach && fs::exists(metadata_path)) + throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Metadata directory {} already exists", metadata_path); + } + else + { + bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + if (create.uuid != UUIDHelpers::Nil && !is_on_cluster) + throw Exception("Ordinary database engine does not support UUID", ErrorCodes::INCORRECT_QUERY); + + /// Ignore UUID if it's ON CLUSTER query + create.uuid = UUIDHelpers::Nil; + metadata_path = metadata_path / "metadata" / database_name_escaped; + } + + DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context); + + if (create.uuid != UUIDHelpers::Nil) + create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER; + + bool need_write_metadata = !create.attach || !fs::exists(metadata_file_path); if (need_write_metadata) { @@ -164,7 +205,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (need_write_metadata) { - Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path); + fs::rename(metadata_file_tmp_path, metadata_file_path); renamed = true; } @@ -173,7 +214,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) catch (...) { if (renamed) - Poco::File(metadata_file_tmp_path).remove(); + { + [[maybe_unused]] bool removed = fs::remove(metadata_file_tmp_path); + assert(removed); + } if (added) DatabaseCatalog::instance().detachDatabase(database_name, false, false); @@ -561,6 +605,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const } } +void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const +{ + const auto * kind = create.is_dictionary ? "Dictionary" : "Table"; + const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE"; + + if (database->getEngineName() == "Atomic") + { + if (create.attach && create.uuid == UUIDHelpers::Nil) + throw Exception(ErrorCodes::INCORRECT_QUERY, + "UUID must be specified in ATTACH {} query for Atomic database engine", + kind_upper); + if (!create.attach && create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); + } + else + { + bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + if (create.uuid != UUIDHelpers::Nil && !is_on_cluster) + throw Exception(ErrorCodes::INCORRECT_QUERY, + "{} UUID specified, but engine of database {} is not Atomic", kind, create.database); + + /// Ignore UUID if it's ON CLUSTER query + create.uuid = UUIDHelpers::Nil; + } +} + BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { @@ -621,19 +691,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (need_add_to_database) { database = DatabaseCatalog::instance().getDatabase(create.database); - if (database->getEngineName() == "Atomic") - { - /// TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/ - if (create.attach && create.uuid == UUIDHelpers::Nil) - throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY); - if (!create.attach && create.uuid == UUIDHelpers::Nil) - create.uuid = UUIDHelpers::generateV4(); - } - else - { - if (create.uuid != UUIDHelpers::Nil) - throw Exception("Table UUID specified, but engine of database " + create.database + " is not Atomic", ErrorCodes::INCORRECT_QUERY); - } + assertOrSetUUID(create, database); /** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing). * If table doesnt exist, one thread is creating table, while others wait in DDLGuard. @@ -662,7 +720,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } data_path = database->getTableDataPath(create); - if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists()) + if (!create.attach && !data_path.empty() && fs::exists(fs::path{context.getPath()} / data_path)) throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS); } else @@ -760,6 +818,12 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create) auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(dictionary_name); create = query->as(); create.attach = true; + } + + assertOrSetUUID(create, database); + + if (create.attach) + { auto config = getDictionaryConfigurationFromAST(create); auto modification_time = database->getObjectMetadataModificationTime(dictionary_name); database->attachDictionary(dictionary_name, DictionaryAttachInfo{query_ptr, config, modification_time}); @@ -775,8 +839,12 @@ BlockIO InterpreterCreateQuery::execute() auto & create = query_ptr->as(); if (!create.cluster.empty()) { - /// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers. - /// However, it allows to use UUID as replica name. + /// Allows to execute ON CLUSTER queries during version upgrade + bool force_backward_compatibility = !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil; + /// For CREATE query generate UUID on initiator, so it will be the same on all hosts. + /// It will be ignored if database does not support UUIDs. + if (!force_backward_compatibility && !create.attach && create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess()); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 30f18aa4134..bb7d70975e4 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -73,6 +73,8 @@ private: /// Inserts data in created table if it's CREATE ... SELECT BlockIO fillTableIfNeeded(const ASTCreateQuery & create); + void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; + ASTPtr query_ptr; Context & context; diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index de2b6bb0c1c..00f6e9b96f8 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -18,23 +17,6 @@ InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, Contex } -struct RenameDescription -{ - RenameDescription(const ASTRenameQuery::Element & elem, const String & current_database) : - from_database_name(elem.from.database.empty() ? current_database : elem.from.database), - from_table_name(elem.from.table), - to_database_name(elem.to.database.empty() ? current_database : elem.to.database), - to_table_name(elem.to.table) - {} - - String from_database_name; - String from_table_name; - - String to_database_name; - String to_table_name; -}; - - BlockIO InterpreterRenameQuery::execute() { const auto & rename = query_ptr->as(); @@ -51,7 +33,7 @@ BlockIO InterpreterRenameQuery::execute() * or we will be in inconsistent state. (It is worth to be fixed.) */ - std::vector descriptions; + RenameDescriptions descriptions; descriptions.reserve(rename.elements.size()); /// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed. @@ -75,22 +57,49 @@ BlockIO InterpreterRenameQuery::execute() for (auto & table_guard : table_guards) table_guard.second = database_catalog.getDDLGuard(table_guard.first.database_name, table_guard.first.table_name); - for (auto & elem : descriptions) + if (rename.database) + return executeToDatabase(rename, descriptions); + else + return executeToTables(rename, descriptions); +} + +BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions) +{ + auto & database_catalog = DatabaseCatalog::instance(); + + for (const auto & elem : descriptions) { if (!rename.exchange) database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), context); database_catalog.getDatabase(elem.from_database_name)->renameTable( - context, - elem.from_table_name, - *database_catalog.getDatabase(elem.to_database_name), - elem.to_table_name, - rename.exchange); + context, + elem.from_table_name, + *database_catalog.getDatabase(elem.to_database_name), + elem.to_table_name, + rename.exchange, + rename.dictionary); } return {}; } +BlockIO InterpreterRenameQuery::executeToDatabase(const ASTRenameQuery &, const RenameDescriptions & descriptions) +{ + assert(descriptions.size() == 1); + assert(descriptions.front().from_table_name.empty()); + assert(descriptions.front().to_table_name.empty()); + + const auto & old_name = descriptions.front().from_database_name; + const auto & new_name = descriptions.back().to_database_name; + auto & catalog = DatabaseCatalog::instance(); + + auto db = catalog.getDatabase(old_name); + catalog.assertDatabaseDoesntExist(new_name); + db->renameDatabase(new_name); + return {}; +} + AccessRightsElements InterpreterRenameQuery::getRequiredAccess() const { AccessRightsElements required_access; @@ -99,6 +108,11 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess() const { required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.from.database, elem.from.table); required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.database, elem.to.table); + if (rename.exchange) + { + required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.database, elem.from.table); + required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.database, elem.to.table); + } } return required_access; } diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index 070601e39d8..f2c08f22602 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -25,6 +25,24 @@ struct UniqueTableName } }; +struct RenameDescription +{ + RenameDescription(const ASTRenameQuery::Element & elem, const String & current_database) : + from_database_name(elem.from.database.empty() ? current_database : elem.from.database), + from_table_name(elem.from.table), + to_database_name(elem.to.database.empty() ? current_database : elem.to.database), + to_table_name(elem.to.table) + {} + + String from_database_name; + String from_table_name; + + String to_database_name; + String to_table_name; +}; + +using RenameDescriptions = std::vector; + using TableGuards = std::map>; /** Rename one table @@ -37,6 +55,9 @@ public: BlockIO execute() override; private: + BlockIO executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions); + static BlockIO executeToDatabase(const ASTRenameQuery & rename, const RenameDescriptions & descriptions); + AccessRightsElements getRequiredAccess() const; ASTPtr query_ptr; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 7c80b681114..4bfa84090c2 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -233,7 +233,8 @@ BlockIO InterpreterSystemQuery::execute() #endif case Type::RELOAD_DICTIONARY: context.checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); - system_context.getExternalDictionariesLoader().loadOrReload(query.target_dictionary); + system_context.getExternalDictionariesLoader().loadOrReload( + DatabaseCatalog::instance().resolveDictionaryName(query.target_dictionary)); ExternalDictionariesLoader::resetAll(); break; case Type::RELOAD_DICTIONARIES: @@ -392,7 +393,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) if (auto table = iterator->table()) { if (dynamic_cast(table.get())) - replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()}); + replica_names.emplace_back(StorageID{iterator->databaseName(), iterator->name()}); } } } diff --git a/src/Interpreters/RowRefs.cpp b/src/Interpreters/RowRefs.cpp index 879a0bcf88e..a206456f1b6 100644 --- a/src/Interpreters/RowRefs.cpp +++ b/src/Interpreters/RowRefs.cpp @@ -12,6 +12,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_TYPE_OF_FIELD; +} + namespace { @@ -56,7 +61,7 @@ AsofRowRefs::AsofRowRefs(TypeIndex type) callWithType(type, call); } -void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Block * block, size_t row_num) +void AsofRowRefs::insert(TypeIndex type, const IColumn & asof_column, const Block * block, size_t row_num) { auto call = [&](const auto & t) { @@ -64,9 +69,9 @@ void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Bloc using LookupPtr = typename Entry::LookupPtr; using ColumnType = std::conditional_t, ColumnDecimal, ColumnVector>; - auto * column = typeid_cast(asof_column); + const auto & column = typeid_cast(asof_column); - T key = column->getElement(row_num); + T key = column.getElement(row_num); auto entry = Entry(key, RowRef(block, row_num)); std::get(lookups)->insert(entry); }; @@ -74,7 +79,7 @@ void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Bloc callWithType(type, call); } -const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const +const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn & asof_column, size_t row_num) const { const RowRef * out = nullptr; @@ -88,8 +93,8 @@ const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality using LookupPtr = typename EntryType::LookupPtr; using ColumnType = std::conditional_t, ColumnDecimal, ColumnVector>; - auto * column = typeid_cast(asof_column); - T key = column->getElement(row_num); + const auto & column = typeid_cast(asof_column); + T key = column.getElement(row_num); auto & typed_lookup = std::get(lookups); if (is_strict) @@ -102,9 +107,9 @@ const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality return out; } -std::optional AsofRowRefs::getTypeSize(const IColumn * asof_column, size_t & size) +std::optional AsofRowRefs::getTypeSize(const IColumn & asof_column, size_t & size) { - TypeIndex idx = asof_column->getDataType(); + TypeIndex idx = asof_column.getDataType(); switch (idx) { @@ -152,8 +157,7 @@ std::optional AsofRowRefs::getTypeSize(const IColumn * asof_column, s break; } - size = 0; - return {}; + throw Exception("ASOF join not supported for type: " + std::string(asof_column.getFamilyName()), ErrorCodes::BAD_TYPE_OF_FIELD); } } diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index e8231b1c233..fc035bf626e 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -233,13 +233,13 @@ public: AsofRowRefs() {} AsofRowRefs(TypeIndex t); - static std::optional getTypeSize(const IColumn * asof_column, size_t & type_size); + static std::optional getTypeSize(const IColumn & asof_column, size_t & type_size); // This will be synchronized by the rwlock mutex in Join.h - void insert(TypeIndex type, const IColumn * asof_column, const Block * block, size_t row_num); + void insert(TypeIndex type, const IColumn & asof_column, const Block * block, size_t row_num); // This will internally synchronize - const RowRef * findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const; + const RowRef * findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn & asof_column, size_t row_num) const; private: // Lookups can be stored in a HashTable because it is memmovable diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index 9bf9047a2dd..96d7b59088d 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -3,7 +3,9 @@ #include #include #include +#include #include +#include namespace DB { @@ -82,4 +84,31 @@ String StorageID::getFullTableName() const return backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name); } +String StorageID::getFullNameNotQuoted() const +{ + return getDatabaseName() + "." + table_name; +} + +StorageID StorageID::fromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, + const String & config_prefix) +{ + StorageID res = StorageID::createEmpty(); + res.database_name = config.getString(config_prefix + ".database", ""); + res.table_name = config.getString(config_prefix + ".name"); + const String uuid_str = config.getString(config_prefix + ".uuid", ""); + if (!uuid_str.empty()) + res.uuid = parseFromString(uuid_str); + return res; +} + +String StorageID::getInternalDictionaryName() const +{ + assertNotEmpty(); + if (hasUUID()) + return toString(uuid); + if (database_name.empty()) + return table_name; + return database_name + "." + table_name; +} + } diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index acbc27a5091..36265b2a2b0 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -6,13 +6,20 @@ #include #include +namespace Poco +{ +namespace Util +{ +class AbstractConfiguration; +} +} + namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; } static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_"; @@ -42,6 +49,7 @@ struct StorageID String getTableName() const; String getFullTableName() const; + String getFullNameNotQuoted() const; String getNameForLogs() const; @@ -66,15 +74,9 @@ struct StorageID { // Can be triggered by user input, e.g. SELECT joinGetOrNull('', 'num', 500) if (empty()) - throw Exception("Table name cannot be empty. Please specify a valid table name or UUID", ErrorCodes::BAD_ARGUMENTS); - - // This can also be triggered by user input, but we haven't decided what - // to do about it: create table "_"(a int) engine Log; - if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID()) - throw Exception("Table name was replaced with placeholder, but UUID is Nil", ErrorCodes::LOGICAL_ERROR); - + throw Exception("Both table name and UUID are empty", ErrorCodes::UNKNOWN_TABLE); if (table_name.empty() && !database_name.empty()) - throw Exception("Table name is empty, but database name is not", ErrorCodes::LOGICAL_ERROR); + throw Exception("Table name is empty, but database name is not", ErrorCodes::UNKNOWN_TABLE); } /// Avoid implicit construction of empty StorageID. However, it's needed for deferred initialization. @@ -82,6 +84,13 @@ struct StorageID QualifiedTableName getQualifiedName() const { return {database_name, getTableName()}; } + static StorageID fromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, + const String & config_prefix); + + /// If dictionary has UUID, then use it as dictionary name in ExternalLoader to allow dictionary renaming. + /// DatabaseCatalog::resolveDictionaryName(...) should be used to access such dictionaries by name. + String getInternalDictionaryName() const; + private: StorageID() = default; }; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 99eaf6e6736..3fdefb3d773 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -131,7 +131,8 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum const auto & dict_name = dict_name_ast->value.safeGet(); const auto & attr_name = attr_name_ast->value.safeGet(); - const auto & dict_ptr = context.getExternalDictionariesLoader().getDictionary(dict_name); + String resolved_name = DatabaseCatalog::instance().resolveDictionaryName(dict_name); + const auto & dict_ptr = context.getExternalDictionariesLoader().getDictionary(resolved_name); if (!dict_ptr->isInjective(attr_name)) { ++i; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 3c8238bc123..991742575b8 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -229,7 +229,6 @@ void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t sub ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query); } - void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const ASTSelectQuery * select_query, const NamesAndTypesList & source_columns, const NameSet & source_columns_set) { diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index bf5701e1be4..0bd97252090 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -17,6 +17,7 @@ #include #include +#include namespace DB @@ -94,9 +95,20 @@ void loadMetadata(Context & context, const String & default_database_name) Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator it(path); it != dir_end; ++it) { - if (!it->isDirectory()) + if (it->isLink()) continue; + if (!it->isDirectory()) + { + if (endsWith(it.name(), ".sql")) + { + String db_name = it.name().substr(0, it.name().size() - 4); + if (db_name != SYSTEM_DATABASE) + databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); + } + continue; + } + /// For '.svn', '.gitignore' directory and similar. if (it.name().at(0) == '.') continue; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 201e2e45528..0df3d17846c 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -205,6 +205,13 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database); + + if (uuid != UUIDHelpers::Nil) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") + << quoteString(toString(uuid)); + } + formatOnCluster(settings); if (storage) @@ -247,6 +254,9 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH " : "CREATE ") << "DICTIONARY " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + if (uuid != UUIDHelpers::Nil) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "") + << quoteString(toString(uuid)); formatOnCluster(settings); } diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index 9f91faffa50..951abbe4419 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -30,6 +30,8 @@ public: Elements elements; bool exchange{false}; /// For EXCHANGE TABLES + bool database{false}; /// For RENAME DATABASE + bool dictionary{false}; /// For RENAME DICTIONARY /** Get the text that identifies this element. */ String getID(char) const override { return "Rename"; } @@ -61,8 +63,24 @@ public: protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") - << (exchange ? "EXCHANGE TABLES " : "RENAME TABLE ") << (settings.hilite ? hilite_none : ""); + if (database) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME DATABASE " << (settings.hilite ? hilite_none : ""); + settings.ostr << backQuoteIfNeed(elements.at(0).from.database); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : ""); + settings.ostr << backQuoteIfNeed(elements.at(0).to.database); + formatOnCluster(settings); + return; + } + + settings.ostr << (settings.hilite ? hilite_keyword : ""); + if (exchange) + settings.ostr << "EXCHANGE TABLES "; + else if (dictionary) + settings.ostr << "RENAME DICTIONARY "; + else + settings.ostr << "RENAME TABLE "; + settings.ostr << (settings.hilite ? hilite_none : ""); for (auto it = elements.cbegin(); it != elements.cend(); ++it) { diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 10874db5a5e..463d2ae8f34 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -595,6 +596,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ASTPtr database; ASTPtr storage; + UUID uuid = UUIDHelpers::Nil; String cluster_str; bool attach = false; @@ -617,6 +619,15 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (!name_p.parse(pos, database, expected)) return false; + if (ParserKeyword("UUID").ignore(pos, expected)) + { + ParserStringLiteral uuid_p; + ASTPtr ast_uuid; + if (!uuid_p.parse(pos, ast_uuid, expected)) + return false; + uuid = parseFromString(ast_uuid->as()->value.get()); + } + if (ParserKeyword{"ON"}.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -633,6 +644,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->if_not_exists = if_not_exists; tryGetIdentifierNameInto(database, query->database); + query->uuid = uuid; query->cluster = cluster_str; query->set(query->storage, storage); @@ -784,7 +796,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E ParserKeyword s_dictionary("DICTIONARY"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_on("ON"); - ParserIdentifier name_p; + ParserCompoundIdentifier dict_name_p(true); ParserToken s_left_paren(TokenType::OpeningRoundBracket); ParserToken s_right_paren(TokenType::ClosingRoundBracket); ParserToken s_dot(TokenType::Dot); @@ -793,7 +805,6 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E bool if_not_exists = false; - ASTPtr database; ASTPtr name; ASTPtr attributes; ASTPtr dictionary; @@ -814,16 +825,9 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E if (s_if_not_exists.ignore(pos, expected)) if_not_exists = true; - if (!name_p.parse(pos, name, expected)) + if (!dict_name_p.parse(pos, name, expected)) return false; - if (s_dot.ignore(pos)) - { - database = name; - if (!name_p.parse(pos, name, expected)) - return false; - } - if (s_on.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) @@ -850,8 +854,10 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E query->is_dictionary = true; query->attach = attach; - tryGetIdentifierNameInto(database, query->database); - tryGetIdentifierNameInto(name, query->table); + StorageID dict_id = getTableIdentifier(name); + query->database = dict_id.database_name; + query->table = dict_id.table_name; + query->uuid = dict_id.uuid; query->if_not_exists = if_not_exists; query->set(query->dictionary_attributes_list, attributes); diff --git a/src/Parsers/ParserRenameQuery.cpp b/src/Parsers/ParserRenameQuery.cpp index abb08b503cd..7fa4e6e5408 100644 --- a/src/Parsers/ParserRenameQuery.cpp +++ b/src/Parsers/ParserRenameQuery.cpp @@ -40,20 +40,52 @@ static bool parseDatabaseAndTable( bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_rename_table("RENAME TABLE"); - ParserKeyword s_to("TO"); ParserKeyword s_exchange_tables("EXCHANGE TABLES"); + ParserKeyword s_rename_dictionary("RENAME DICTIONARY"); + ParserKeyword s_rename_database("RENAME DATABASE"); + ParserKeyword s_to("TO"); ParserKeyword s_and("AND"); ParserToken s_comma(TokenType::Comma); bool exchange = false; + bool dictionary = false; - if (!s_rename_table.ignore(pos, expected)) + if (s_rename_table.ignore(pos, expected)) + ; + else if (s_exchange_tables.ignore(pos, expected)) + exchange = true; + else if (s_rename_dictionary.ignore(pos, expected)) + dictionary = true; + else if (s_rename_database.ignore(pos, expected)) { - if (s_exchange_tables.ignore(pos, expected)) - exchange = true; - else + ASTPtr from_db; + ASTPtr to_db; + ParserIdentifier db_name_p; + if (!db_name_p.parse(pos, from_db, expected)) return false; + if (!s_to.ignore(pos, expected)) + return false; + if (!db_name_p.parse(pos, to_db, expected)) + return false; + + String cluster_str; + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto query = std::make_shared(); + query->database = true; + query->elements.emplace({}); + tryGetIdentifierNameInto(from_db, query->elements.front().from.database); + tryGetIdentifierNameInto(to_db, query->elements.front().to.database); + query->cluster = cluster_str; + node = query; + return true; } + else + return false; ASTRenameQuery::Elements elements; @@ -88,6 +120,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->elements = elements; query->exchange = exchange; + query->dictionary = dictionary; return true; } diff --git a/src/Processors/Sources/DelayedSource.cpp b/src/Processors/Sources/DelayedSource.cpp index 42a33d00196..f9e112f5e68 100644 --- a/src/Processors/Sources/DelayedSource.cpp +++ b/src/Processors/Sources/DelayedSource.cpp @@ -1,13 +1,30 @@ #include -#include "NullSource.h" +#include +#include namespace DB { -DelayedSource::DelayedSource(const Block & header, Creator processors_creator) - : IProcessor({}, OutputPorts(3, header)) +DelayedSource::DelayedSource(const Block & header, Creator processors_creator, bool add_totals_port, bool add_extremes_port) + : IProcessor({}, OutputPorts(1 + (add_totals_port ? 1 : 0) + (add_extremes_port ? 1 : 0), header)) , creator(std::move(processors_creator)) { + auto output = outputs.begin(); + + main = &*output; + ++output; + + if (add_totals_port) + { + totals = &*output; + ++output; + } + + if (add_extremes_port) + { + extremes = &*output; + ++output; + } } IProcessor::Status DelayedSource::prepare() @@ -66,6 +83,31 @@ IProcessor::Status DelayedSource::prepare() return Status::Finished; } +/// Fix port from returned pipe. Create source_port if created or drop if source_port is null. +void synchronizePorts(OutputPort *& pipe_port, OutputPort * source_port, const Block & header, Processors & processors) +{ + if (source_port) + { + /// Need port in DelayedSource. Create NullSource. + if (!pipe_port) + { + processors.emplace_back(std::make_shared(header)); + pipe_port = &processors.back()->getOutputs().back(); + } + } + else + { + /// Has port in pipe, but don't need it. Create NullSink. + if (pipe_port) + { + auto sink = std::make_shared(header); + connect(*pipe_port, sink->getPort()); + processors.emplace_back(std::move(sink)); + pipe_port = nullptr; + } + } +} + void DelayedSource::work() { auto pipe = creator(); @@ -76,17 +118,8 @@ void DelayedSource::work() processors = std::move(pipe).detachProcessors(); - if (!totals_output) - { - processors.emplace_back(std::make_shared(main_output->getHeader())); - totals_output = &processors.back()->getOutputs().back(); - } - - if (!extremes_output) - { - processors.emplace_back(std::make_shared(main_output->getHeader())); - extremes_output = &processors.back()->getOutputs().back(); - } + synchronizePorts(totals_output, totals, main->getHeader(), processors); + synchronizePorts(extremes_output, extremes, main->getHeader(), processors); } Processors DelayedSource::expandPipeline() @@ -94,6 +127,9 @@ Processors DelayedSource::expandPipeline() /// Add new inputs. They must have the same header as output. for (const auto & output : {main_output, totals_output, extremes_output}) { + if (!output) + continue; + inputs.emplace_back(outputs.front().getHeader(), this); /// Connect checks that header is same for ports. connect(*output, inputs.back()); @@ -104,13 +140,13 @@ Processors DelayedSource::expandPipeline() return std::move(processors); } -Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator) +Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator, bool add_totals_port, bool add_extremes_port) { - auto source = std::make_shared(header, std::move(processors_creator)); + auto source = std::make_shared(header, std::move(processors_creator), add_totals_port, add_extremes_port); - Pipe pipe(&source->getPort(DelayedSource::Main)); - pipe.setTotalsPort(&source->getPort(DelayedSource::Totals)); - pipe.setExtremesPort(&source->getPort(DelayedSource::Extremes)); + Pipe pipe(&source->getPort()); + pipe.setTotalsPort(source->getTotalsPort()); + pipe.setExtremesPort(source->getExtremesPort()); pipe.addProcessors({std::move(source)}); return pipe; diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 31ec1e054fe..7c2b104b61e 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -19,20 +19,26 @@ class DelayedSource : public IProcessor public: using Creator = std::function; - DelayedSource(const Block & header, Creator processors_creator); + DelayedSource(const Block & header, Creator processors_creator, bool add_totals_port, bool add_extremes_port); String getName() const override { return "Delayed"; } Status prepare() override; void work() override; Processors expandPipeline() override; - enum PortKind { Main = 0, Totals = 1, Extremes = 2 }; - OutputPort & getPort(PortKind kind) { return *std::next(outputs.begin(), kind); } + OutputPort & getPort() { return *main; } + OutputPort * getTotalsPort() { return totals; } + OutputPort * getExtremesPort() { return extremes; } private: Creator creator; Processors processors; + /// Outputs for DelayedSource. + OutputPort * main = nullptr; + OutputPort * totals = nullptr; + OutputPort * extremes = nullptr; + /// Outputs from returned pipe. OutputPort * main_output = nullptr; OutputPort * totals_output = nullptr; @@ -40,6 +46,6 @@ private: }; /// Creates pipe from DelayedSource. -Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator); +Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator, bool add_totals_port, bool add_extremes_port); } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index ce1f669d70f..2d49419abe3 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -169,7 +169,7 @@ SettingsChanges StorageKafka::createSettingsAdjustments() if (!schema_name.empty()) result.emplace_back("format_schema", schema_name); - for (auto setting : *kafka_settings) + for (const auto & setting : *kafka_settings) { const auto & name = setting.getName(); if (name.find("kafka_") == std::string::npos) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d4f1aea14c7..06e03e9da5e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -146,6 +146,10 @@ MergeTreeData::MergeTreeData( if (relative_data_path.empty()) throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); + /// Check sanity of MergeTreeSettings. Only when table is created. + if (!attach) + settings->sanityCheck(global_context.getSettingsRef()); + MergeTreeDataFormatVersion min_format_version(0); if (!date_column_name.empty()) { @@ -1608,6 +1612,7 @@ void MergeTreeData::changeSettings( const auto & new_changes = new_settings->as().changes; for (const auto & change : new_changes) + { if (change.name == "storage_policy") { StoragePolicyPtr new_storage_policy = global_context.getStoragePolicy(change.value.safeGet()); @@ -1642,9 +1647,13 @@ void MergeTreeData::changeSettings( has_storage_policy_changed = true; } } + } MergeTreeSettings copy = *getSettings(); copy.applyChanges(new_changes); + + copy.sanityCheck(global_context.getSettingsRef()); + storage_settings.set(std::make_unique(copy)); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); new_metadata.setSettingsChanges(new_settings); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 75a41218efc..81142655ab4 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -75,4 +75,31 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #undef ADD_IF_ABSENT } +void MergeTreeSettings::sanityCheck(const Settings & query_settings) const +{ + if (number_of_free_entries_in_pool_to_execute_mutation >= query_settings.background_pool_size) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_mutation' setting" + " ({}) (default values are defined in section of config.xml" + " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" + " is greater or equals to the value of 'background_pool_size'" + " ({}) (the value is defined in users.xml for default profile)." + " This indicates incorrect configuration because mutations cannot work with these settings.", + number_of_free_entries_in_pool_to_execute_mutation, + query_settings.background_pool_size); + } + + if (number_of_free_entries_in_pool_to_lower_max_size_of_merge >= query_settings.background_pool_size) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting" + " ({}) (default values are defined in section of config.xml" + " or the value can be specified per table in SETTINGS section of CREATE TABLE query)" + " is greater or equals to the value of 'background_pool_size'" + " ({}) (the value is defined in users.xml for default profile)." + " This indicates incorrect configuration because the maximum size of merge will be always lowered.", + number_of_free_entries_in_pool_to_execute_mutation, + query_settings.background_pool_size); + } +} + } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b8307ee91d5..9675c6e303c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -13,6 +13,7 @@ namespace Poco::Util namespace DB { class ASTStorage; +struct Settings; #define LIST_OF_MERGE_TREE_SETTINGS(M) \ @@ -123,6 +124,9 @@ struct MergeTreeSettings : public BaseSettings return name == "min_bytes_for_wide_part" || name == "min_rows_for_wide_part" || name == "min_bytes_for_compact_part" || name == "min_rows_for_compact_part"; } + + /// Check that the values are sane taking also query-level settings into account. + void sanityCheck(const Settings & query_settings) const; }; using MergeTreeSettingsPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index aa9d405130f..b7881694565 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -32,6 +33,7 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; + extern const int LOGICAL_ERROR; } @@ -324,8 +326,16 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - add_mandatory_param("path in ZooKeeper"); - add_mandatory_param("replica name"); + if (is_extended_storage_def) + { + add_optional_param("path in ZooKeeper"); + add_optional_param("replica name"); + } + else + { + add_mandatory_param("path in ZooKeeper"); + add_mandatory_param("replica name"); + } } if (!is_extended_storage_def) @@ -394,28 +404,50 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - const auto * ast = engine_args[arg_num]->as(); - if (ast && ast->value.getType() == Field::Types::String) - zookeeper_path = safeGet(ast->value); - else - throw Exception( - "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), - ErrorCodes::BAD_ARGUMENTS); - ++arg_num; + bool has_arguments = arg_num + 2 <= arg_cnt + && engine_args[arg_num]->as() + && engine_args[arg_num + 1]->as(); - ast = engine_args[arg_num]->as(); - if (ast && ast->value.getType() == Field::Types::String) - replica_name = safeGet(ast->value); - else - throw Exception( - "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), - ErrorCodes::BAD_ARGUMENTS); + if (has_arguments) + { + const auto * ast = engine_args[arg_num]->as(); + if (ast && ast->value.getType() == Field::Types::String) + zookeeper_path = safeGet(ast->value); + else + throw Exception( + "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), + ErrorCodes::BAD_ARGUMENTS); + ++arg_num; - if (replica_name.empty()) - throw Exception( - "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), - ErrorCodes::NO_REPLICA_NAME_GIVEN); - ++arg_num; + ast = engine_args[arg_num]->as(); + if (ast && ast->value.getType() == Field::Types::String) + replica_name = safeGet(ast->value); + else + throw Exception( + "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), + ErrorCodes::BAD_ARGUMENTS); + + if (replica_name.empty()) + throw Exception( + "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), + ErrorCodes::NO_REPLICA_NAME_GIVEN); + ++arg_num; + } + else if (is_extended_storage_def) + { + /// Try use default values if arguments are not specified. + /// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config. + zookeeper_path = "/clickhouse/tables/{uuid}/{shard}"; + replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined? + } + else + throw Exception("Expected zookeper_path and replica_name arguments", ErrorCodes::LOGICAL_ERROR); + + /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries + bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + bool allow_uuid_macro = is_on_cluster || args.query.attach; + zookeeper_path = args.context.getMacros()->expand(zookeeper_path, args.table_id, allow_uuid_macro); + replica_name = args.context.getMacros()->expand(replica_name, args.table_id, false); } /// This merging param maybe used as part of sorting key diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a88d2d4af4d..59e3ca4cdc8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -187,8 +187,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( true, /// require_part_metadata attach, [this] (const std::string & name) { enqueuePartForCheck(name); }) - , zookeeper_path(normalizeZooKeeperPath(global_context.getMacros()->expand(zookeeper_path_, table_id_.database_name, table_id_.table_name))) - , replica_name(global_context.getMacros()->expand(replica_name_, table_id_.database_name, table_id_.table_name)) + , zookeeper_path(normalizeZooKeeperPath(zookeeper_path_)) + , replica_name(replica_name_) , replica_path(zookeeper_path + "/replicas/" + replica_name) , reader(*this) , writer(*this) diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 5a35e079a5b..2a89bf34186 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -15,6 +16,7 @@ NamesAndTypesList StorageSystemDatabases::getNamesAndTypes() {"engine", std::make_shared()}, {"data_path", std::make_shared()}, {"metadata_path", std::make_shared()}, + {"uuid", std::make_shared()}, }; } @@ -33,6 +35,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Contex res_columns[1]->insert(database.second->getEngineName()); res_columns[2]->insert(context.getPath() + database.second->getDataPath()); res_columns[3]->insert(database.second->getMetadataPath()); + res_columns[4]->insert(database.second->getUUID()); } } diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index 4c54353c44d..6661f51b02f 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -25,6 +26,7 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() return { {"database", std::make_shared()}, {"name", std::make_shared()}, + {"uuid", std::make_shared()}, {"status", std::make_shared(getStatusEnumAllPossibleValues())}, {"origin", std::make_shared()}, {"type", std::make_shared()}, @@ -57,30 +59,23 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con { const auto dict_ptr = std::dynamic_pointer_cast(load_result.object); - String database, short_name; + StorageID dict_id = StorageID::createEmpty(); if (dict_ptr) - { - database = dict_ptr->getDatabase(); - short_name = dict_ptr->getName(); - } + dict_id = dict_ptr->getDictionaryID(); + else if (load_result.config) + dict_id = StorageID::fromDictionaryConfig(*load_result.config->config, load_result.config->key_in_config); else - { - short_name = load_result.name; - String repository_name = load_result.config ? load_result.config->repository_name : ""; - if (!repository_name.empty() && startsWith(short_name, repository_name + ".")) - { - database = repository_name; - short_name = short_name.substr(database.length() + 1); - } - } + dict_id.table_name = load_result.name; + String db_or_tag = dict_id.database_name.empty() ? IDictionary::NO_DATABASE_TAG : dict_id.database_name; if (check_access_for_dictionaries - && !access->isGranted(AccessType::SHOW_DICTIONARIES, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name)) + && !access->isGranted(AccessType::SHOW_DICTIONARIES, db_or_tag, dict_id.table_name)) continue; size_t i = 0; - res_columns[i++]->insert(database); - res_columns[i++]->insert(short_name); + res_columns[i++]->insert(dict_id.database_name); + res_columns[i++]->insert(dict_id.table_name); + res_columns[i++]->insert(dict_id.uuid); res_columns[i++]->insert(static_cast(load_result.status)); res_columns[i++]->insert(load_result.config ? load_result.config->path : ""); diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 5e94a2382f7..4de600ac036 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -20,7 +20,7 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - for (auto setting : context.getMergeTreeSettings().all()) + for (const auto & setting : context.getMergeTreeSettings().all()) { res_columns[0]->insert(setting.getName()); res_columns[1]->insert(setting.getValueString()); diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index 44af7d0588d..07a2d450b12 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -30,7 +30,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context { const Settings & settings = context.getSettingsRef(); auto settings_constraints = context.getSettingsConstraints(); - for (auto setting : settings.all()) + for (const auto & setting : settings.all()) { const auto & setting_name = setting.getName(); res_columns[0]->insert(setting_name); diff --git a/tests/config/macros.xml b/tests/config/macros.xml index 1f86f5f9efd..97c3065471f 100644 --- a/tests/config/macros.xml +++ b/tests/config/macros.xml @@ -1,5 +1,7 @@ Hello, world! + s1 + r1 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b6a97b2fb6f..f421f979947 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -345,7 +345,7 @@ class ClickHouseCluster: container_id = self.get_container_id(instance_name) return self.docker_client.api.logs(container_id) - def exec_in_container(self, container_id, cmd, detach=False, **kwargs): + def exec_in_container(self, container_id, cmd, detach=False, nothrow=False, **kwargs): exec_id = self.docker_client.api.exec_create(container_id, cmd, **kwargs) output = self.docker_client.api.exec_start(exec_id, detach=detach) @@ -361,7 +361,11 @@ class ClickHouseCluster: print("Container {} uses image {}: ".format(container_id, image_id)) pprint.pprint(image_info) print("") - raise Exception('Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output)) + message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output) + if nothrow: + print(message) + else: + raise Exception(message) return output def copy_file_to_container(self, container_id, local_path, dest_path): @@ -888,9 +892,9 @@ class ClickHouseInstance: from helpers.test_tools import assert_eq_with_retry assert_eq_with_retry(self, "select 1", "1", retry_count=int(stop_start_wait_sec / 0.5), sleep_time=0.5) - def exec_in_container(self, cmd, detach=False, **kwargs): + def exec_in_container(self, cmd, detach=False, nothrow=False, **kwargs): container_id = self.get_docker_handle().id - return self.cluster.exec_in_container(container_id, cmd, detach, **kwargs) + return self.cluster.exec_in_container(container_id, cmd, detach, nothrow, **kwargs) def contains_in_log(self, substring): result = self.exec_in_container( @@ -928,7 +932,8 @@ class ClickHouseInstance: # force kill if server hangs if self.get_process_pid("clickhouse server"): - self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(9)], user='root') + # server can die before kill, so don't throw exception, it's expected + self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(9)], nothrow=True, user='root') if callback_onstop: callback_onstop(self) @@ -1191,4 +1196,3 @@ class ClickHouseKiller(object): def __exit__(self, exc_type, exc_val, exc_tb): self.clickhouse_node.restore_clickhouse() - diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 9c2bcc22ef7..983cac596dc 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -78,7 +78,7 @@ class Task1: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) - ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format(cluster_num)) + ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(cluster_num)) ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " + @@ -115,7 +115,7 @@ class Task2: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) - ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format(cluster_num)) + ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(cluster_num)) ddl_check_query(instance, "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") ddl_check_query(instance, "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)") diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index c0966f77deb..70c66653cb2 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -62,7 +62,7 @@ class TaskTrivial: for node in [source, destination]: node.query("DROP DATABASE IF EXISTS default") - node.query("CREATE DATABASE IF NOT EXISTS default") + node.query("CREATE DATABASE IF NOT EXISTS default ENGINE=Ordinary") source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " @@ -181,4 +181,4 @@ if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: for name, instance in cluster.instances.items(): print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") \ No newline at end of file + raw_input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index 66a3a9e1a34..882cd566472 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -69,6 +69,12 @@ SELECT sum(x) FROM distributed SETTINGS max_replica_delay_for_distributed_queries=1 ''').strip() == '3' + assert instance_with_dist_table.query(''' +SELECT sum(x) FROM distributed WITH TOTALS SETTINGS + load_balancing='in_order', + max_replica_delay_for_distributed_queries=1 +''').strip() == '3\n\n3' + pm.drop_instance_zk_connections(node_1_2) pm.drop_instance_zk_connections(node_2_2) diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index 28071381586..082a76cd88d 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -68,8 +68,8 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): assert len(set(codes)) == 1, "\n" + tsv_content assert codes[0] == "0", "\n" + tsv_content - def ddl_check_query(self, instance, query, num_hosts=None): - contents = instance.query(query) + def ddl_check_query(self, instance, query, num_hosts=None, settings=None): + contents = instance.query(query, settings=settings) self.check_all_hosts_successfully_executed(contents, num_hosts) return contents diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 8ef6f1892c5..f5dbe0ef8d2 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -295,6 +295,22 @@ def test_socket_timeout(test_cluster): for i in range(0, 100): instance.query("select hostName() as host, count() from cluster('cluster', 'system', 'settings') group by host") +def test_replicated_without_arguments(test_cluster): + rules = test_cluster.pm_random_drops.pop_rules() + instance = test_cluster.instances['ch1'] + test_cluster.ddl_check_query(instance, "CREATE DATABASE test_atomic ON CLUSTER cluster ENGINE=Atomic", + settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + test_cluster.ddl_check_query(instance, "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", + settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster") + test_cluster.ddl_check_query(instance, "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", + settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster") + test_cluster.ddl_check_query(instance, "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", + settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + test_cluster.ddl_check_query(instance, "EXCHANGE TABLES test_atomic.rmt AND test_atomic.rmt_renamed ON CLUSTER cluster") + test_cluster.pm_random_drops.push_rules(rules) + if __name__ == '__main__': with contextmanager(test_cluster)() as ctx_cluster: for name, instance in ctx_cluster.instances.items(): diff --git a/tests/integration/test_insert_into_distributed/configs/forbid_background_merges.xml b/tests/integration/test_insert_into_distributed/configs/forbid_background_merges.xml index ffdca0cf6bc..bc2dae31ad6 100644 --- a/tests/integration/test_insert_into_distributed/configs/forbid_background_merges.xml +++ b/tests/integration/test_insert_into_distributed/configs/forbid_background_merges.xml @@ -1,7 +1,6 @@ - - - 0 - - + + 1 + 2 + diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index e48584bac84..731ffbbe2fd 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -12,7 +12,7 @@ instance_test_reconnect = cluster.add_instance('instance_test_reconnect', main_c instance_test_inserts_batching = cluster.add_instance( 'instance_test_inserts_batching', main_configs=['configs/remote_servers.xml'], user_configs=['configs/enable_distributed_inserts_batching.xml']) -remote = cluster.add_instance('remote', user_configs=['configs/forbid_background_merges.xml']) +remote = cluster.add_instance('remote', main_configs=['configs/forbid_background_merges.xml']) instance_test_inserts_local_cluster = cluster.add_instance( 'instance_test_inserts_local_cluster', diff --git a/tests/integration/test_insert_into_distributed_through_materialized_view/configs/forbid_background_merges.xml b/tests/integration/test_insert_into_distributed_through_materialized_view/configs/forbid_background_merges.xml index ffdca0cf6bc..bc2dae31ad6 100644 --- a/tests/integration/test_insert_into_distributed_through_materialized_view/configs/forbid_background_merges.xml +++ b/tests/integration/test_insert_into_distributed_through_materialized_view/configs/forbid_background_merges.xml @@ -1,7 +1,6 @@ - - - 0 - - + + 1 + 2 + diff --git a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py index 2dc8d7326dd..1df803920f1 100644 --- a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py +++ b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py @@ -12,7 +12,7 @@ instance_test_reconnect = cluster.add_instance('instance_test_reconnect', main_c instance_test_inserts_batching = cluster.add_instance( 'instance_test_inserts_batching', main_configs=['configs/remote_servers.xml'], user_configs=['configs/enable_distributed_inserts_batching.xml']) -remote = cluster.add_instance('remote', user_configs=['configs/forbid_background_merges.xml']) +remote = cluster.add_instance('remote', main_configs=['configs/forbid_background_merges.xml']) instance_test_inserts_local_cluster = cluster.add_instance( 'instance_test_inserts_local_cluster', diff --git a/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml b/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml index 8b57af4f48e..82aaeb1fbc8 100644 --- a/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml +++ b/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml @@ -2,7 +2,6 @@ 1 2 - 100 0 diff --git a/tests/integration/test_storage_kafka/configs/kafka_macros.xml b/tests/integration/test_storage_kafka/configs/kafka_macros.xml new file mode 100644 index 00000000000..7f6cfb5eb1f --- /dev/null +++ b/tests/integration/test_storage_kafka/configs/kafka_macros.xml @@ -0,0 +1,13 @@ + + + + kafka1 + old + old + + new + new + instance + JSONEachRow + + \ No newline at end of file diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 938c7e2b1b4..ca50c435069 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -35,7 +35,7 @@ import kafka_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', config_dir='configs', - main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], + main_configs=['configs/kafka.xml', 'configs/log_conf.xml', 'configs/kafka_macros.xml' ], with_kafka=True, with_zookeeper=True, clickhouse_path_dir='clickhouse_path') @@ -570,9 +570,18 @@ def kafka_setup_teardown(): @pytest.mark.timeout(180) def test_kafka_settings_old_syntax(kafka_cluster): + assert TSV(instance.query("SELECT * FROM system.macros WHERE macro like 'kafka%' ORDER BY macro", ignore_error=True)) == TSV('''kafka_broker kafka1 +kafka_client_id instance +kafka_format_json_each_row JSONEachRow +kafka_group_name_new new +kafka_group_name_old old +kafka_topic_new new +kafka_topic_old old +''') + instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('kafka1:19092', 'old', 'old', 'JSONEachRow', '\\n'); + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); ''') # Don't insert malformed messages since old settings syntax @@ -599,12 +608,12 @@ def test_kafka_settings_new_syntax(kafka_cluster): instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'new', - kafka_group_name = 'new', - kafka_format = 'JSONEachRow', + SETTINGS kafka_broker_list = '{kafka_broker}:19092', + kafka_topic_list = '{kafka_topic_new}', + kafka_group_name = '{kafka_group_name_new}', + kafka_format = '{kafka_format_json_each_row}', kafka_row_delimiter = '\\n', - kafka_client_id = '{instance} test 1234', + kafka_client_id = '{kafka_client_id} test 1234', kafka_skip_broken_messages = 1; ''') diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 1894f88029e..eedcb01ee3a 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -97,7 +97,7 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): with pytest.raises(QueryRuntimeException): node1.query(get_command("TTL d1 TO DISK 'unknown'", "small_jbod_with_external")) - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) if alter: node1.query(get_command(None, "small_jbod_with_external")) @@ -105,7 +105,7 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): with pytest.raises(QueryRuntimeException): node1.query(get_command("TTL d1 TO VOLUME 'unknown'", "small_jbod_with_external")) - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) if alter: node1.query(get_command(None, "only_jbod2")) @@ -113,7 +113,7 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): with pytest.raises(QueryRuntimeException): node1.query(get_command("TTL d1 TO DISK 'jbod1'", "only_jbod2")) - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) if alter: node1.query(get_command(None, "only_jbod2")) @@ -122,7 +122,7 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): node1.query(get_command("TTL d1 TO VOLUME 'external'", "only_jbod2")) finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine,positive", [ @@ -155,7 +155,7 @@ def test_inserts_to_disk_work(started_cluster, name, engine, positive): finally: try: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) except: pass @@ -204,7 +204,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine,positive", [ @@ -250,7 +250,7 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine", [ @@ -296,7 +296,7 @@ def test_moves_to_volume_work(started_cluster, name, engine): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine,positive", [ @@ -334,7 +334,7 @@ def test_inserts_to_volume_work(started_cluster, name, engine, positive): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "20" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine", [ @@ -379,7 +379,7 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod1"} - node1.query("DROP TABLE {}".format(name_temp)) + node1.query("DROP TABLE {} NO DELAY".format(name_temp)) time.sleep(2) used_disks = get_used_disks_for_table(node1, name) @@ -388,8 +388,8 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name_temp)) - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name_temp)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) def test_replicated_download_ttl_info(started_cluster): @@ -420,7 +420,7 @@ def test_replicated_download_ttl_info(started_cluster): finally: for node in (node1, node2): try: - node.query("DROP TABLE IF EXISTS {}".format(name)) + node.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) except: continue @@ -479,7 +479,7 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "16" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine", [ @@ -544,8 +544,8 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "12" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name_temp)) - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name_temp)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine,positive", [ @@ -597,7 +597,7 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "14" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine,positive,bar", [ @@ -640,7 +640,7 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, b assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == "10" finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine", [ @@ -702,7 +702,7 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): assert node1.query("SELECT count() FROM {name}".format(name=name)).strip() == str(len(data)) finally: - node1.query("DROP TABLE IF EXISTS {}".format(name)) + node1.query("DROP TABLE IF EXISTS {} NO DELAY".format(name)) @pytest.mark.parametrize("name,engine,positive", [ @@ -799,7 +799,7 @@ limitations under the License.""" assert rows_count == 3 finally: - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + node1.query("DROP TABLE IF EXISTS {name} NO DELAY".format(name=name)) @pytest.mark.parametrize("name,engine", [ @@ -897,7 +897,7 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): assert node1.query("SELECT 1") == "1\n" assert node1.query("SELECT COUNT() FROM {}".format(name)) == "500\n" finally: - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + node1.query("DROP TABLE IF EXISTS {name} NO DELAY".format(name=name)) @pytest.mark.skip(reason="Flacky test") @pytest.mark.parametrize("name,positive", [ @@ -950,7 +950,7 @@ def test_double_move_while_select(started_cluster, name, positive): assert node1.query("SELECT n FROM {name} ORDER BY n".format(name=name)).splitlines() == ["1", "2", "3", "4"] finally: - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + node1.query("DROP TABLE IF EXISTS {name} NO DELAY".format(name=name)) @pytest.mark.parametrize("name,engine,positive", [ @@ -1040,4 +1040,4 @@ limitations under the License.""" assert node1.query("SELECT count() FROM {name}".format(name=name)) == "6\n" finally: - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + node1.query("DROP TABLE IF EXISTS {name} NO DELAY".format(name=name)) diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index fb28429c37c..468e92c7aac 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -37,6 +37,11 @@ +SELECT number % 10 AS k FROM numbers(100000000) GROUP BY k FORMAT Null +SELECT number % 10 AS k FROM numbers_mt(1600000000) GROUP BY k FORMAT Null +SELECT number % 256 AS k FROM numbers(100000000) GROUP BY k FORMAT Null +SELECT number % 256 AS k FROM numbers_mt(1600000000) GROUP BY k FORMAT Null + SELECT number % 10 AS k, count() FROM numbers(100000000) GROUP BY k FORMAT Null SELECT number % 10 AS k, count() FROM numbers_mt(1600000000) GROUP BY k FORMAT Null SELECT number % 256 AS k, count() FROM numbers(100000000) GROUP BY k FORMAT Null @@ -46,19 +51,28 @@ SELECT number % 10 AS k, count(), sum(number), avg(number) FROM numbers_mt(1600000000) GROUP BY k FORMAT Null SELECT number % 256 AS k, count(), sum(number), avg(number) FROM numbers(100000000) GROUP BY k FORMAT Null SELECT number % 256 AS k, count(), sum(number), avg(number) FROM numbers_mt(1600000000) GROUP BY k FORMAT Null +SELECT number % 256 AS k, count(), sum(number), avg(number), min(number), max(number), uniq(number), any(number), argMin(number, number), argMax(number, number) FROM numbers_mt(160000000) GROUP BY k FORMAT Null SELECT number % 1000 AS k, count() FROM numbers( 100000000) GROUP BY k FORMAT Null SELECT number % 1000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k FORMAT Null +SELECT number % 1000 AS k FROM numbers( 100000000) GROUP BY k FORMAT Null +SELECT number % 1000 AS k FROM numbers_mt(1600000000) GROUP BY k FORMAT Null SELECT number % 100000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null SELECT number % 100000 AS k, count() FROM numbers_mt(160000000) GROUP BY k FORMAT Null +SELECT number % 100000 AS k FROM numbers( 10000000) GROUP BY k FORMAT Null +SELECT number % 100000 AS k FROM numbers_mt(160000000) GROUP BY k FORMAT Null SELECT number % 1000000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null SELECT number % 1000000 AS k, count() FROM numbers_mt(160000000) GROUP BY k FORMAT Null +SELECT number % 1000000 AS k FROM numbers( 10000000) GROUP BY k FORMAT Null +SELECT number % 1000000 AS k FROM numbers_mt(160000000) GROUP BY k FORMAT Null SELECT number % 10000000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null SELECT number % 10000000 AS k, count() FROM numbers_mt(80000000) GROUP BY k FORMAT Null +SELECT number % 10000000 AS k FROM numbers( 10000000) GROUP BY k FORMAT Null +SELECT number % 10000000 AS k FROM numbers_mt(80000000) GROUP BY k FORMAT Null true @@ -36,9 +58,23 @@ 9440 1 + + + + + + clickhouse2 + 9000 + - - + + + clickhouse3 + 9000 + + + + clickhouse1 @@ -81,5 +117,32 @@ + + + true + + clickhouse1 + 9440 + 1 + + user_with_a_very_very_very_very_long_name_we_will_use_him_to_simulate_the_problem_with_inserting_to_distributed_when_folder_name_is_too_long_i_hope_that_length_is_enough_username_end + + + + clickhouse2 + 9440 + 1 + user_with_a_very_very_very_very_long_name_we_will_use_him_to_simulate_the_problem_with_inserting_to_distributed_when_folder_name_is_too_long_i_hope_that_length_is_enough_username_end + + + + clickhouse3 + 9440 + 1 + user_with_a_very_very_very_very_long_name_we_will_use_him_to_simulate_the_problem_with_inserting_to_distributed_when_folder_name_is_too_long_i_hope_that_length_is_enough_username_end + + + + diff --git a/tests/testflows/rbac/regression.py b/tests/testflows/rbac/regression.py index 6a96183298c..46bccf917fb 100755 --- a/tests/testflows/rbac/regression.py +++ b/tests/testflows/rbac/regression.py @@ -2,7 +2,7 @@ import sys from testflows.core import * -append_path(sys.path, "..") +append_path(sys.path, "..") from helpers.cluster import Cluster from helpers.argparser import argparser @@ -56,6 +56,7 @@ def regression(self, local, clickhouse_binary_path): self.context.cluster = cluster Feature(run=load("rbac.tests.syntax.feature", "feature"), flags=TE) + Feature(run=load("rbac.tests.privileges.feature", "feature"), flags=TE) if main(): regression() diff --git a/tests/testflows/rbac/requirements/requirements.md b/tests/testflows/rbac/requirements/requirements.md index 5590853c9a0..def2fd258e1 100644 --- a/tests/testflows/rbac/requirements/requirements.md +++ b/tests/testflows/rbac/requirements/requirements.md @@ -34,408 +34,415 @@ * 5.1.4.4 [RQ.SRS-006.RBAC.Role.SettingsProfile](#rqsrs-006rbacrolesettingsprofile) * 5.1.4.5 [RQ.SRS-006.RBAC.Role.Quotas](#rqsrs-006rbacrolequotas) * 5.1.4.6 [RQ.SRS-006.RBAC.Role.RowPolicies](#rqsrs-006rbacrolerowpolicies) - * 5.1.5 [Privileges](#privileges) - * 5.1.5.1 [RQ.SRS-006.RBAC.Privileges.Usage](#rqsrs-006rbacprivilegesusage) - * 5.1.5.2 [RQ.SRS-006.RBAC.Privileges.Select](#rqsrs-006rbacprivilegesselect) - * 5.1.5.3 [RQ.SRS-006.RBAC.Privileges.SelectColumns](#rqsrs-006rbacprivilegesselectcolumns) - * 5.1.5.4 [RQ.SRS-006.RBAC.Privileges.Insert](#rqsrs-006rbacprivilegesinsert) - * 5.1.5.5 [RQ.SRS-006.RBAC.Privileges.Delete](#rqsrs-006rbacprivilegesdelete) - * 5.1.5.6 [RQ.SRS-006.RBAC.Privileges.Alter](#rqsrs-006rbacprivilegesalter) - * 5.1.5.7 [RQ.SRS-006.RBAC.Privileges.Create](#rqsrs-006rbacprivilegescreate) - * 5.1.5.8 [RQ.SRS-006.RBAC.Privileges.Drop](#rqsrs-006rbacprivilegesdrop) - * 5.1.5.9 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.1.5.10 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.1.5.11 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.1.5.12 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) - * 5.1.6 [Required Privileges](#required-privileges) - * 5.1.6.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Insert](#rqsrs-006rbacrequiredprivilegesinsert) - * 5.1.6.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Select](#rqsrs-006rbacrequiredprivilegesselect) - * 5.1.6.3 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) - * 5.1.6.4 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) - * 5.1.6.5 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop](#rqsrs-006rbacrequiredprivilegesdrop) - * 5.1.6.6 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table](#rqsrs-006rbacrequiredprivilegesdroptable) - * 5.1.6.7 [RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke](#rqsrs-006rbacrequiredprivilegesgrantrevoke) - * 5.1.6.8 [RQ.SRS-006.RBAC.RequiredPrivileges.Use](#rqsrs-006rbacrequiredprivilegesuse) - * 5.1.6.9 [RQ.SRS-006.RBAC.RequiredPrivileges.Admin](#rqsrs-006rbacrequiredprivilegesadmin) - * 5.1.7 [Partial Revokes](#partial-revokes) - * 5.1.7.1 [RQ.SRS-006.RBAC.PartialRevokes](#rqsrs-006rbacpartialrevokes) - * 5.1.8 [Settings Profile](#settings-profile) - * 5.1.8.1 [RQ.SRS-006.RBAC.SettingsProfile](#rqsrs-006rbacsettingsprofile) - * 5.1.8.2 [RQ.SRS-006.RBAC.SettingsProfile.Constraints](#rqsrs-006rbacsettingsprofileconstraints) - * 5.1.8.3 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreate](#rqsrs-006rbacsettingsprofileshowcreate) - * 5.1.9 [Quotas](#quotas) - * 5.1.9.1 [RQ.SRS-006.RBAC.Quotas](#rqsrs-006rbacquotas) - * 5.1.9.2 [RQ.SRS-006.RBAC.Quotas.Keyed](#rqsrs-006rbacquotaskeyed) - * 5.1.9.3 [RQ.SRS-006.RBAC.Quotas.Queries](#rqsrs-006rbacquotasqueries) - * 5.1.9.4 [RQ.SRS-006.RBAC.Quotas.Errors](#rqsrs-006rbacquotaserrors) - * 5.1.9.5 [RQ.SRS-006.RBAC.Quotas.ResultRows](#rqsrs-006rbacquotasresultrows) - * 5.1.9.6 [RQ.SRS-006.RBAC.Quotas.ReadRows](#rqsrs-006rbacquotasreadrows) - * 5.1.9.7 [RQ.SRS-006.RBAC.Quotas.ResultBytes](#rqsrs-006rbacquotasresultbytes) - * 5.1.9.8 [RQ.SRS-006.RBAC.Quotas.ReadBytes](#rqsrs-006rbacquotasreadbytes) - * 5.1.9.9 [RQ.SRS-006.RBAC.Quotas.ExecutionTime](#rqsrs-006rbacquotasexecutiontime) - * 5.1.9.10 [RQ.SRS-006.RBAC.Quotas.ShowCreate](#rqsrs-006rbacquotasshowcreate) - * 5.1.10 [Row Policy](#row-policy) - * 5.1.10.1 [RQ.SRS-006.RBAC.RowPolicy](#rqsrs-006rbacrowpolicy) - * 5.1.10.2 [RQ.SRS-006.RBAC.RowPolicy.Condition](#rqsrs-006rbacrowpolicycondition) - * 5.1.10.3 [RQ.SRS-006.RBAC.RowPolicy.ShowCreate](#rqsrs-006rbacrowpolicyshowcreate) + * 5.1.5 [Partial Revokes](#partial-revokes) + * 5.1.5.1 [RQ.SRS-006.RBAC.PartialRevokes](#rqsrs-006rbacpartialrevokes) + * 5.1.6 [Settings Profile](#settings-profile) + * 5.1.6.1 [RQ.SRS-006.RBAC.SettingsProfile](#rqsrs-006rbacsettingsprofile) + * 5.1.6.2 [RQ.SRS-006.RBAC.SettingsProfile.Constraints](#rqsrs-006rbacsettingsprofileconstraints) + * 5.1.6.3 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreate](#rqsrs-006rbacsettingsprofileshowcreate) + * 5.1.7 [Quotas](#quotas) + * 5.1.7.1 [RQ.SRS-006.RBAC.Quotas](#rqsrs-006rbacquotas) + * 5.1.7.2 [RQ.SRS-006.RBAC.Quotas.Keyed](#rqsrs-006rbacquotaskeyed) + * 5.1.7.3 [RQ.SRS-006.RBAC.Quotas.Queries](#rqsrs-006rbacquotasqueries) + * 5.1.7.4 [RQ.SRS-006.RBAC.Quotas.Errors](#rqsrs-006rbacquotaserrors) + * 5.1.7.5 [RQ.SRS-006.RBAC.Quotas.ResultRows](#rqsrs-006rbacquotasresultrows) + * 5.1.7.6 [RQ.SRS-006.RBAC.Quotas.ReadRows](#rqsrs-006rbacquotasreadrows) + * 5.1.7.7 [RQ.SRS-006.RBAC.Quotas.ResultBytes](#rqsrs-006rbacquotasresultbytes) + * 5.1.7.8 [RQ.SRS-006.RBAC.Quotas.ReadBytes](#rqsrs-006rbacquotasreadbytes) + * 5.1.7.9 [RQ.SRS-006.RBAC.Quotas.ExecutionTime](#rqsrs-006rbacquotasexecutiontime) + * 5.1.7.10 [RQ.SRS-006.RBAC.Quotas.ShowCreate](#rqsrs-006rbacquotasshowcreate) + * 5.1.8 [Row Policy](#row-policy) + * 5.1.8.1 [RQ.SRS-006.RBAC.RowPolicy](#rqsrs-006rbacrowpolicy) + * 5.1.8.2 [RQ.SRS-006.RBAC.RowPolicy.Condition](#rqsrs-006rbacrowpolicycondition) + * 5.1.8.3 [RQ.SRS-006.RBAC.RowPolicy.ShowCreate](#rqsrs-006rbacrowpolicyshowcreate) * 5.2 [Specific](#specific) - * 5.2.10.1 [RQ.SRS-006.RBAC.User.Use.DefaultRole](#rqsrs-006rbacuserusedefaultrole) - * 5.2.10.2 [RQ.SRS-006.RBAC.User.Use.AllRolesWhenNoDefaultRole](#rqsrs-006rbacuseruseallroleswhennodefaultrole) - * 5.2.10.3 [RQ.SRS-006.RBAC.User.Create](#rqsrs-006rbacusercreate) - * 5.2.10.4 [RQ.SRS-006.RBAC.User.Create.IfNotExists](#rqsrs-006rbacusercreateifnotexists) - * 5.2.10.5 [RQ.SRS-006.RBAC.User.Create.Replace](#rqsrs-006rbacusercreatereplace) - * 5.2.10.6 [RQ.SRS-006.RBAC.User.Create.Password.NoPassword](#rqsrs-006rbacusercreatepasswordnopassword) - * 5.2.10.7 [RQ.SRS-006.RBAC.User.Create.Password.NoPassword.Login](#rqsrs-006rbacusercreatepasswordnopasswordlogin) - * 5.2.10.8 [RQ.SRS-006.RBAC.User.Create.Password.PlainText](#rqsrs-006rbacusercreatepasswordplaintext) - * 5.2.10.9 [RQ.SRS-006.RBAC.User.Create.Password.PlainText.Login](#rqsrs-006rbacusercreatepasswordplaintextlogin) - * 5.2.10.10 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Password](#rqsrs-006rbacusercreatepasswordsha256password) - * 5.2.10.11 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Password.Login](#rqsrs-006rbacusercreatepasswordsha256passwordlogin) - * 5.2.10.12 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash](#rqsrs-006rbacusercreatepasswordsha256hash) - * 5.2.10.13 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash.Login](#rqsrs-006rbacusercreatepasswordsha256hashlogin) - * 5.2.10.14 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password](#rqsrs-006rbacusercreatepassworddoublesha1password) - * 5.2.10.15 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password.Login](#rqsrs-006rbacusercreatepassworddoublesha1passwordlogin) - * 5.2.10.16 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash](#rqsrs-006rbacusercreatepassworddoublesha1hash) - * 5.2.10.17 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash.Login](#rqsrs-006rbacusercreatepassworddoublesha1hashlogin) - * 5.2.10.18 [RQ.SRS-006.RBAC.User.Create.Host.Name](#rqsrs-006rbacusercreatehostname) - * 5.2.10.19 [RQ.SRS-006.RBAC.User.Create.Host.Regexp](#rqsrs-006rbacusercreatehostregexp) - * 5.2.10.20 [RQ.SRS-006.RBAC.User.Create.Host.IP](#rqsrs-006rbacusercreatehostip) - * 5.2.10.21 [RQ.SRS-006.RBAC.User.Create.Host.Any](#rqsrs-006rbacusercreatehostany) - * 5.2.10.22 [RQ.SRS-006.RBAC.User.Create.Host.None](#rqsrs-006rbacusercreatehostnone) - * 5.2.10.23 [RQ.SRS-006.RBAC.User.Create.Host.Local](#rqsrs-006rbacusercreatehostlocal) - * 5.2.10.24 [RQ.SRS-006.RBAC.User.Create.Host.Like](#rqsrs-006rbacusercreatehostlike) - * 5.2.10.25 [RQ.SRS-006.RBAC.User.Create.Host.Default](#rqsrs-006rbacusercreatehostdefault) - * 5.2.10.26 [RQ.SRS-006.RBAC.User.Create.DefaultRole](#rqsrs-006rbacusercreatedefaultrole) - * 5.2.10.27 [RQ.SRS-006.RBAC.User.Create.DefaultRole.None](#rqsrs-006rbacusercreatedefaultrolenone) - * 5.2.10.28 [RQ.SRS-006.RBAC.User.Create.DefaultRole.All](#rqsrs-006rbacusercreatedefaultroleall) - * 5.2.10.29 [RQ.SRS-006.RBAC.User.Create.Settings](#rqsrs-006rbacusercreatesettings) - * 5.2.10.30 [RQ.SRS-006.RBAC.User.Create.OnCluster](#rqsrs-006rbacusercreateoncluster) - * 5.2.10.31 [RQ.SRS-006.RBAC.User.Create.Syntax](#rqsrs-006rbacusercreatesyntax) - * 5.2.10.32 [RQ.SRS-006.RBAC.User.Alter](#rqsrs-006rbacuseralter) - * 5.2.10.33 [RQ.SRS-006.RBAC.User.Alter.OrderOfEvaluation](#rqsrs-006rbacuseralterorderofevaluation) - * 5.2.10.34 [RQ.SRS-006.RBAC.User.Alter.IfExists](#rqsrs-006rbacuseralterifexists) - * 5.2.10.35 [RQ.SRS-006.RBAC.User.Alter.Cluster](#rqsrs-006rbacuseraltercluster) - * 5.2.10.36 [RQ.SRS-006.RBAC.User.Alter.Rename](#rqsrs-006rbacuseralterrename) - * 5.2.10.37 [RQ.SRS-006.RBAC.User.Alter.Password.PlainText](#rqsrs-006rbacuseralterpasswordplaintext) - * 5.2.10.38 [RQ.SRS-006.RBAC.User.Alter.Password.Sha256Password](#rqsrs-006rbacuseralterpasswordsha256password) - * 5.2.10.39 [RQ.SRS-006.RBAC.User.Alter.Password.DoubleSha1Password](#rqsrs-006rbacuseralterpassworddoublesha1password) - * 5.2.10.40 [RQ.SRS-006.RBAC.User.Alter.Host.AddDrop](#rqsrs-006rbacuseralterhostadddrop) - * 5.2.10.41 [RQ.SRS-006.RBAC.User.Alter.Host.Local](#rqsrs-006rbacuseralterhostlocal) - * 5.2.10.42 [RQ.SRS-006.RBAC.User.Alter.Host.Name](#rqsrs-006rbacuseralterhostname) - * 5.2.10.43 [RQ.SRS-006.RBAC.User.Alter.Host.Regexp](#rqsrs-006rbacuseralterhostregexp) - * 5.2.10.44 [RQ.SRS-006.RBAC.User.Alter.Host.IP](#rqsrs-006rbacuseralterhostip) - * 5.2.10.45 [RQ.SRS-006.RBAC.User.Alter.Host.Like](#rqsrs-006rbacuseralterhostlike) - * 5.2.10.46 [RQ.SRS-006.RBAC.User.Alter.Host.Any](#rqsrs-006rbacuseralterhostany) - * 5.2.10.47 [RQ.SRS-006.RBAC.User.Alter.Host.None](#rqsrs-006rbacuseralterhostnone) - * 5.2.10.48 [RQ.SRS-006.RBAC.User.Alter.DefaultRole](#rqsrs-006rbacuseralterdefaultrole) - * 5.2.10.49 [RQ.SRS-006.RBAC.User.Alter.DefaultRole.All](#rqsrs-006rbacuseralterdefaultroleall) - * 5.2.10.50 [RQ.SRS-006.RBAC.User.Alter.DefaultRole.AllExcept](#rqsrs-006rbacuseralterdefaultroleallexcept) - * 5.2.10.51 [RQ.SRS-006.RBAC.User.Alter.Settings](#rqsrs-006rbacuseraltersettings) - * 5.2.10.52 [RQ.SRS-006.RBAC.User.Alter.Settings.Min](#rqsrs-006rbacuseraltersettingsmin) - * 5.2.10.53 [RQ.SRS-006.RBAC.User.Alter.Settings.Max](#rqsrs-006rbacuseraltersettingsmax) - * 5.2.10.54 [RQ.SRS-006.RBAC.User.Alter.Settings.Profile](#rqsrs-006rbacuseraltersettingsprofile) - * 5.2.10.55 [RQ.SRS-006.RBAC.User.Alter.Syntax](#rqsrs-006rbacuseraltersyntax) - * 5.2.10.56 [RQ.SRS-006.RBAC.SetDefaultRole](#rqsrs-006rbacsetdefaultrole) - * 5.2.10.57 [RQ.SRS-006.RBAC.SetDefaultRole.CurrentUser](#rqsrs-006rbacsetdefaultrolecurrentuser) - * 5.2.10.58 [RQ.SRS-006.RBAC.SetDefaultRole.All](#rqsrs-006rbacsetdefaultroleall) - * 5.2.10.59 [RQ.SRS-006.RBAC.SetDefaultRole.AllExcept](#rqsrs-006rbacsetdefaultroleallexcept) - * 5.2.10.60 [RQ.SRS-006.RBAC.SetDefaultRole.None](#rqsrs-006rbacsetdefaultrolenone) - * 5.2.10.61 [RQ.SRS-006.RBAC.SetDefaultRole.Syntax](#rqsrs-006rbacsetdefaultrolesyntax) - * 5.2.10.62 [RQ.SRS-006.RBAC.SetRole](#rqsrs-006rbacsetrole) - * 5.2.10.63 [RQ.SRS-006.RBAC.SetRole.Default](#rqsrs-006rbacsetroledefault) - * 5.2.10.64 [RQ.SRS-006.RBAC.SetRole.None](#rqsrs-006rbacsetrolenone) - * 5.2.10.65 [RQ.SRS-006.RBAC.SetRole.All](#rqsrs-006rbacsetroleall) - * 5.2.10.66 [RQ.SRS-006.RBAC.SetRole.AllExcept](#rqsrs-006rbacsetroleallexcept) - * 5.2.10.67 [RQ.SRS-006.RBAC.SetRole.Syntax](#rqsrs-006rbacsetrolesyntax) - * 5.2.10.68 [RQ.SRS-006.RBAC.User.ShowCreateUser](#rqsrs-006rbacusershowcreateuser) - * 5.2.10.69 [RQ.SRS-006.RBAC.User.ShowCreateUser.For](#rqsrs-006rbacusershowcreateuserfor) - * 5.2.10.70 [RQ.SRS-006.RBAC.User.ShowCreateUser.Syntax](#rqsrs-006rbacusershowcreateusersyntax) - * 5.2.10.71 [RQ.SRS-006.RBAC.User.Drop](#rqsrs-006rbacuserdrop) - * 5.2.10.72 [RQ.SRS-006.RBAC.User.Drop.IfExists](#rqsrs-006rbacuserdropifexists) - * 5.2.10.73 [RQ.SRS-006.RBAC.User.Drop.OnCluster](#rqsrs-006rbacuserdroponcluster) - * 5.2.10.74 [RQ.SRS-006.RBAC.User.Drop.Syntax](#rqsrs-006rbacuserdropsyntax) - * 5.2.10.75 [RQ.SRS-006.RBAC.Role.Create](#rqsrs-006rbacrolecreate) - * 5.2.10.76 [RQ.SRS-006.RBAC.Role.Create.IfNotExists](#rqsrs-006rbacrolecreateifnotexists) - * 5.2.10.77 [RQ.SRS-006.RBAC.Role.Create.Replace](#rqsrs-006rbacrolecreatereplace) - * 5.2.10.78 [RQ.SRS-006.RBAC.Role.Create.Settings](#rqsrs-006rbacrolecreatesettings) - * 5.2.10.79 [RQ.SRS-006.RBAC.Role.Create.Syntax](#rqsrs-006rbacrolecreatesyntax) - * 5.2.10.80 [RQ.SRS-006.RBAC.Role.Create.Effect](#rqsrs-006rbacrolecreateeffect) - * 5.2.10.81 [RQ.SRS-006.RBAC.Role.Alter](#rqsrs-006rbacrolealter) - * 5.2.10.82 [RQ.SRS-006.RBAC.Role.Alter.IfExists](#rqsrs-006rbacrolealterifexists) - * 5.2.10.83 [RQ.SRS-006.RBAC.Role.Alter.Cluster](#rqsrs-006rbacrolealtercluster) - * 5.2.10.84 [RQ.SRS-006.RBAC.Role.Alter.Rename](#rqsrs-006rbacrolealterrename) - * 5.2.10.85 [RQ.SRS-006.RBAC.Role.Alter.Settings](#rqsrs-006rbacrolealtersettings) - * 5.2.10.86 [RQ.SRS-006.RBAC.Role.Alter.Effect](#rqsrs-006rbacrolealtereffect) - * 5.2.10.87 [RQ.SRS-006.RBAC.Role.Alter.Syntax](#rqsrs-006rbacrolealtersyntax) - * 5.2.10.88 [RQ.SRS-006.RBAC.Role.Drop](#rqsrs-006rbacroledrop) - * 5.2.10.89 [RQ.SRS-006.RBAC.Role.Drop.IfExists](#rqsrs-006rbacroledropifexists) - * 5.2.10.90 [RQ.SRS-006.RBAC.Role.Drop.Cluster](#rqsrs-006rbacroledropcluster) - * 5.2.10.91 [RQ.SRS-006.RBAC.Role.Drop.Effect](#rqsrs-006rbacroledropeffect) - * 5.2.10.92 [RQ.SRS-006.RBAC.Role.Drop.Syntax](#rqsrs-006rbacroledropsyntax) - * 5.2.10.93 [RQ.SRS-006.RBAC.Role.ShowCreate](#rqsrs-006rbacroleshowcreate) - * 5.2.10.94 [RQ.SRS-006.RBAC.Role.ShowCreate.Syntax](#rqsrs-006rbacroleshowcreatesyntax) - * 5.2.10.95 [RQ.SRS-006.RBAC.Grant.Privilege.To](#rqsrs-006rbacgrantprivilegeto) - * 5.2.10.96 [RQ.SRS-006.RBAC.Grant.Privilege.To.Effect](#rqsrs-006rbacgrantprivilegetoeffect) - * 5.2.10.97 [RQ.SRS-006.RBAC.Grant.Privilege.ToCurrentUser](#rqsrs-006rbacgrantprivilegetocurrentuser) - * 5.2.10.98 [RQ.SRS-006.RBAC.Grant.Privilege.Select](#rqsrs-006rbacgrantprivilegeselect) - * 5.2.10.99 [RQ.SRS-006.RBAC.Grant.Privilege.Select.Effect](#rqsrs-006rbacgrantprivilegeselecteffect) - * 5.2.10.100 [RQ.SRS-006.RBAC.Grant.Privilege.SelectColumns](#rqsrs-006rbacgrantprivilegeselectcolumns) - * 5.2.10.101 [RQ.SRS-006.RBAC.Grant.Privilege.SelectColumns.Effect](#rqsrs-006rbacgrantprivilegeselectcolumnseffect) - * 5.2.10.102 [RQ.SRS-006.RBAC.Grant.Privilege.Insert](#rqsrs-006rbacgrantprivilegeinsert) - * 5.2.10.103 [RQ.SRS-006.RBAC.Grant.Privilege.Insert.Effect](#rqsrs-006rbacgrantprivilegeinserteffect) - * 5.2.10.104 [RQ.SRS-006.RBAC.Grant.Privilege.Alter](#rqsrs-006rbacgrantprivilegealter) - * 5.2.10.105 [RQ.SRS-006.RBAC.Grant.Privilege.Alter.Effect](#rqsrs-006rbacgrantprivilegealtereffect) - * 5.2.10.106 [RQ.SRS-006.RBAC.Grant.Privilege.Create](#rqsrs-006rbacgrantprivilegecreate) - * 5.2.10.107 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Effect](#rqsrs-006rbacgrantprivilegecreateeffect) - * 5.2.10.108 [RQ.SRS-006.RBAC.Grant.Privilege.Drop](#rqsrs-006rbacgrantprivilegedrop) - * 5.2.10.109 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Effect](#rqsrs-006rbacgrantprivilegedropeffect) - * 5.2.10.110 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate](#rqsrs-006rbacgrantprivilegetruncate) - * 5.2.10.111 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate.Effect](#rqsrs-006rbacgrantprivilegetruncateeffect) - * 5.2.10.112 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize](#rqsrs-006rbacgrantprivilegeoptimize) - * 5.2.10.113 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize.Effect](#rqsrs-006rbacgrantprivilegeoptimizeeffect) - * 5.2.10.114 [RQ.SRS-006.RBAC.Grant.Privilege.Show](#rqsrs-006rbacgrantprivilegeshow) - * 5.2.10.115 [RQ.SRS-006.RBAC.Grant.Privilege.Show.Effect](#rqsrs-006rbacgrantprivilegeshoweffect) - * 5.2.10.116 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery](#rqsrs-006rbacgrantprivilegekillquery) - * 5.2.10.117 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery.Effect](#rqsrs-006rbacgrantprivilegekillqueryeffect) - * 5.2.10.118 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement](#rqsrs-006rbacgrantprivilegeaccessmanagement) - * 5.2.10.119 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement.Effect](#rqsrs-006rbacgrantprivilegeaccessmanagementeffect) - * 5.2.10.120 [RQ.SRS-006.RBAC.Grant.Privilege.System](#rqsrs-006rbacgrantprivilegesystem) - * 5.2.10.121 [RQ.SRS-006.RBAC.Grant.Privilege.System.Effect](#rqsrs-006rbacgrantprivilegesystemeffect) - * 5.2.10.122 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection](#rqsrs-006rbacgrantprivilegeintrospection) - * 5.2.10.123 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection.Effect](#rqsrs-006rbacgrantprivilegeintrospectioneffect) - * 5.2.10.124 [RQ.SRS-006.RBAC.Grant.Privilege.Sources](#rqsrs-006rbacgrantprivilegesources) - * 5.2.10.125 [RQ.SRS-006.RBAC.Grant.Privilege.Sources.Effect](#rqsrs-006rbacgrantprivilegesourceseffect) - * 5.2.10.126 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet](#rqsrs-006rbacgrantprivilegedictget) - * 5.2.10.127 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet.Effect](#rqsrs-006rbacgrantprivilegedictgeteffect) - * 5.2.10.128 [RQ.SRS-006.RBAC.Grant.Privilege.None](#rqsrs-006rbacgrantprivilegenone) - * 5.2.10.129 [RQ.SRS-006.RBAC.Grant.Privilege.None.Effect](#rqsrs-006rbacgrantprivilegenoneeffect) - * 5.2.10.130 [RQ.SRS-006.RBAC.Grant.Privilege.All](#rqsrs-006rbacgrantprivilegeall) - * 5.2.10.131 [RQ.SRS-006.RBAC.Grant.Privilege.All.Effect](#rqsrs-006rbacgrantprivilegealleffect) - * 5.2.10.132 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption](#rqsrs-006rbacgrantprivilegegrantoption) - * 5.2.10.133 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption.Effect](#rqsrs-006rbacgrantprivilegegrantoptioneffect) - * 5.2.10.134 [RQ.SRS-006.RBAC.Grant.Privilege.On](#rqsrs-006rbacgrantprivilegeon) - * 5.2.10.135 [RQ.SRS-006.RBAC.Grant.Privilege.On.Effect](#rqsrs-006rbacgrantprivilegeoneffect) - * 5.2.10.136 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns](#rqsrs-006rbacgrantprivilegeprivilegecolumns) - * 5.2.10.137 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns.Effect](#rqsrs-006rbacgrantprivilegeprivilegecolumnseffect) - * 5.2.10.138 [RQ.SRS-006.RBAC.Grant.Privilege.OnCluster](#rqsrs-006rbacgrantprivilegeoncluster) - * 5.2.10.139 [RQ.SRS-006.RBAC.Grant.Privilege.Syntax](#rqsrs-006rbacgrantprivilegesyntax) - * 5.2.10.140 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster](#rqsrs-006rbacrevokeprivilegecluster) - * 5.2.10.141 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster.Effect](#rqsrs-006rbacrevokeprivilegeclustereffect) - * 5.2.10.142 [RQ.SRS-006.RBAC.Revoke.Privilege.Any](#rqsrs-006rbacrevokeprivilegeany) - * 5.2.10.143 [RQ.SRS-006.RBAC.Revoke.Privilege.Any.Effect](#rqsrs-006rbacrevokeprivilegeanyeffect) - * 5.2.10.144 [RQ.SRS-006.RBAC.Revoke.Privilege.Select](#rqsrs-006rbacrevokeprivilegeselect) - * 5.2.10.145 [RQ.SRS-006.RBAC.Revoke.Privilege.Select.Effect](#rqsrs-006rbacrevokeprivilegeselecteffect) - * 5.2.10.146 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert](#rqsrs-006rbacrevokeprivilegeinsert) - * 5.2.10.147 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert.Effect](#rqsrs-006rbacrevokeprivilegeinserteffect) - * 5.2.10.148 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter](#rqsrs-006rbacrevokeprivilegealter) - * 5.2.10.149 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter.Effect](#rqsrs-006rbacrevokeprivilegealtereffect) - * 5.2.10.150 [RQ.SRS-006.RBAC.Revoke.Privilege.Create](#rqsrs-006rbacrevokeprivilegecreate) - * 5.2.10.151 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Effect](#rqsrs-006rbacrevokeprivilegecreateeffect) - * 5.2.10.152 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop](#rqsrs-006rbacrevokeprivilegedrop) - * 5.2.10.153 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Effect](#rqsrs-006rbacrevokeprivilegedropeffect) - * 5.2.10.154 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate](#rqsrs-006rbacrevokeprivilegetruncate) - * 5.2.10.155 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate.Effect](#rqsrs-006rbacrevokeprivilegetruncateeffect) - * 5.2.10.156 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize](#rqsrs-006rbacrevokeprivilegeoptimize) - * 5.2.10.157 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize.Effect](#rqsrs-006rbacrevokeprivilegeoptimizeeffect) - * 5.2.10.158 [RQ.SRS-006.RBAC.Revoke.Privilege.Show](#rqsrs-006rbacrevokeprivilegeshow) - * 5.2.10.159 [RQ.SRS-006.RBAC.Revoke.Privilege.Show.Effect](#rqsrs-006rbacrevokeprivilegeshoweffect) - * 5.2.10.160 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery](#rqsrs-006rbacrevokeprivilegekillquery) - * 5.2.10.161 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery.Effect](#rqsrs-006rbacrevokeprivilegekillqueryeffect) - * 5.2.10.162 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement](#rqsrs-006rbacrevokeprivilegeaccessmanagement) - * 5.2.10.163 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement.Effect](#rqsrs-006rbacrevokeprivilegeaccessmanagementeffect) - * 5.2.10.164 [RQ.SRS-006.RBAC.Revoke.Privilege.System](#rqsrs-006rbacrevokeprivilegesystem) - * 5.2.10.165 [RQ.SRS-006.RBAC.Revoke.Privilege.System.Effect](#rqsrs-006rbacrevokeprivilegesystemeffect) - * 5.2.10.166 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection](#rqsrs-006rbacrevokeprivilegeintrospection) - * 5.2.10.167 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection.Effect](#rqsrs-006rbacrevokeprivilegeintrospectioneffect) - * 5.2.10.168 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources](#rqsrs-006rbacrevokeprivilegesources) - * 5.2.10.169 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources.Effect](#rqsrs-006rbacrevokeprivilegesourceseffect) - * 5.2.10.170 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet](#rqsrs-006rbacrevokeprivilegedictget) - * 5.2.10.171 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet.Effect](#rqsrs-006rbacrevokeprivilegedictgeteffect) - * 5.2.10.172 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns](#rqsrs-006rbacrevokeprivilegeprivelegecolumns) - * 5.2.10.173 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns.Effect](#rqsrs-006rbacrevokeprivilegeprivelegecolumnseffect) - * 5.2.10.174 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple](#rqsrs-006rbacrevokeprivilegemultiple) - * 5.2.10.175 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple.Effect](#rqsrs-006rbacrevokeprivilegemultipleeffect) - * 5.2.10.176 [RQ.SRS-006.RBAC.Revoke.Privilege.All](#rqsrs-006rbacrevokeprivilegeall) - * 5.2.10.177 [RQ.SRS-006.RBAC.Revoke.Privilege.All.Effect](#rqsrs-006rbacrevokeprivilegealleffect) - * 5.2.10.178 [RQ.SRS-006.RBAC.Revoke.Privilege.None](#rqsrs-006rbacrevokeprivilegenone) - * 5.2.10.179 [RQ.SRS-006.RBAC.Revoke.Privilege.None.Effect](#rqsrs-006rbacrevokeprivilegenoneeffect) - * 5.2.10.180 [RQ.SRS-006.RBAC.Revoke.Privilege.On](#rqsrs-006rbacrevokeprivilegeon) - * 5.2.10.181 [RQ.SRS-006.RBAC.Revoke.Privilege.On.Effect](#rqsrs-006rbacrevokeprivilegeoneffect) - * 5.2.10.182 [RQ.SRS-006.RBAC.Revoke.Privilege.From](#rqsrs-006rbacrevokeprivilegefrom) - * 5.2.10.183 [RQ.SRS-006.RBAC.Revoke.Privilege.From.Effect](#rqsrs-006rbacrevokeprivilegefromeffect) - * 5.2.10.184 [RQ.SRS-006.RBAC.Revoke.Privilege.Syntax](#rqsrs-006rbacrevokeprivilegesyntax) - * 5.2.10.185 [RQ.SRS-006.RBAC.PartialRevoke.Syntax](#rqsrs-006rbacpartialrevokesyntax) - * 5.2.10.186 [RQ.SRS-006.RBAC.PartialRevoke.Effect](#rqsrs-006rbacpartialrevokeeffect) - * 5.2.10.187 [RQ.SRS-006.RBAC.Grant.Role](#rqsrs-006rbacgrantrole) - * 5.2.10.188 [RQ.SRS-006.RBAC.Grant.Role.Effect](#rqsrs-006rbacgrantroleeffect) - * 5.2.10.189 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser](#rqsrs-006rbacgrantrolecurrentuser) - * 5.2.10.190 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser.Effect](#rqsrs-006rbacgrantrolecurrentusereffect) - * 5.2.10.191 [RQ.SRS-006.RBAC.Grant.Role.AdminOption](#rqsrs-006rbacgrantroleadminoption) - * 5.2.10.192 [RQ.SRS-006.RBAC.Grant.Role.AdminOption.Effect](#rqsrs-006rbacgrantroleadminoptioneffect) - * 5.2.10.193 [RQ.SRS-006.RBAC.Grant.Role.OnCluster](#rqsrs-006rbacgrantroleoncluster) - * 5.2.10.194 [RQ.SRS-006.RBAC.Grant.Role.Syntax](#rqsrs-006rbacgrantrolesyntax) - * 5.2.10.195 [RQ.SRS-006.RBAC.Revoke.Role](#rqsrs-006rbacrevokerole) - * 5.2.10.196 [RQ.SRS-006.RBAC.Revoke.Role.Effect](#rqsrs-006rbacrevokeroleeffect) - * 5.2.10.197 [RQ.SRS-006.RBAC.Revoke.Role.Keywords](#rqsrs-006rbacrevokerolekeywords) - * 5.2.10.198 [RQ.SRS-006.RBAC.Revoke.Role.Keywords.Effect](#rqsrs-006rbacrevokerolekeywordseffect) - * 5.2.10.199 [RQ.SRS-006.RBAC.Revoke.Role.Cluster](#rqsrs-006rbacrevokerolecluster) - * 5.2.10.200 [RQ.SRS-006.RBAC.Revoke.Role.Cluster.Effect](#rqsrs-006rbacrevokeroleclustereffect) - * 5.2.10.201 [RQ.SRS-006.RBAC.Revoke.AdminOption](#rqsrs-006rbacrevokeadminoption) - * 5.2.10.202 [RQ.SRS-006.RBAC.Revoke.AdminOption.Effect](#rqsrs-006rbacrevokeadminoptioneffect) - * 5.2.10.203 [RQ.SRS-006.RBAC.Revoke.Role.Syntax](#rqsrs-006rbacrevokerolesyntax) - * 5.2.10.204 [RQ.SRS-006.RBAC.Show.Grants](#rqsrs-006rbacshowgrants) - * 5.2.10.205 [RQ.SRS-006.RBAC.Show.Grants.For](#rqsrs-006rbacshowgrantsfor) - * 5.2.10.206 [RQ.SRS-006.RBAC.Show.Grants.Syntax](#rqsrs-006rbacshowgrantssyntax) - * 5.2.10.207 [RQ.SRS-006.RBAC.SettingsProfile.Create](#rqsrs-006rbacsettingsprofilecreate) - * 5.2.10.208 [RQ.SRS-006.RBAC.SettingsProfile.Create.Effect](#rqsrs-006rbacsettingsprofilecreateeffect) - * 5.2.10.209 [RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists](#rqsrs-006rbacsettingsprofilecreateifnotexists) - * 5.2.10.210 [RQ.SRS-006.RBAC.SettingsProfile.Create.Replace](#rqsrs-006rbacsettingsprofilecreatereplace) - * 5.2.10.211 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables](#rqsrs-006rbacsettingsprofilecreatevariables) - * 5.2.10.212 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value](#rqsrs-006rbacsettingsprofilecreatevariablesvalue) - * 5.2.10.213 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value.Effect](#rqsrs-006rbacsettingsprofilecreatevariablesvalueeffect) - * 5.2.10.214 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints](#rqsrs-006rbacsettingsprofilecreatevariablesconstraints) - * 5.2.10.215 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints.Effect](#rqsrs-006rbacsettingsprofilecreatevariablesconstraintseffect) - * 5.2.10.216 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment](#rqsrs-006rbacsettingsprofilecreateassignment) - * 5.2.10.217 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None](#rqsrs-006rbacsettingsprofilecreateassignmentnone) - * 5.2.10.218 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All](#rqsrs-006rbacsettingsprofilecreateassignmentall) - * 5.2.10.219 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilecreateassignmentallexcept) - * 5.2.10.220 [RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit](#rqsrs-006rbacsettingsprofilecreateinherit) - * 5.2.10.221 [RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster](#rqsrs-006rbacsettingsprofilecreateoncluster) - * 5.2.10.222 [RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax](#rqsrs-006rbacsettingsprofilecreatesyntax) - * 5.2.10.223 [RQ.SRS-006.RBAC.SettingsProfile.Alter](#rqsrs-006rbacsettingsprofilealter) - * 5.2.10.224 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Effect](#rqsrs-006rbacsettingsprofilealtereffect) - * 5.2.10.225 [RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists](#rqsrs-006rbacsettingsprofilealterifexists) - * 5.2.10.226 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename](#rqsrs-006rbacsettingsprofilealterrename) - * 5.2.10.227 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables](#rqsrs-006rbacsettingsprofilealtervariables) - * 5.2.10.228 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value](#rqsrs-006rbacsettingsprofilealtervariablesvalue) - * 5.2.10.229 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value.Effect](#rqsrs-006rbacsettingsprofilealtervariablesvalueeffect) - * 5.2.10.230 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints](#rqsrs-006rbacsettingsprofilealtervariablesconstraints) - * 5.2.10.231 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints.Effect](#rqsrs-006rbacsettingsprofilealtervariablesconstraintseffect) - * 5.2.10.232 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment](#rqsrs-006rbacsettingsprofilealterassignment) - * 5.2.10.233 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Effect](#rqsrs-006rbacsettingsprofilealterassignmenteffect) - * 5.2.10.234 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None](#rqsrs-006rbacsettingsprofilealterassignmentnone) - * 5.2.10.235 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All](#rqsrs-006rbacsettingsprofilealterassignmentall) - * 5.2.10.236 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilealterassignmentallexcept) - * 5.2.10.237 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit](#rqsrs-006rbacsettingsprofilealterassignmentinherit) - * 5.2.10.238 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster](#rqsrs-006rbacsettingsprofilealterassignmentoncluster) - * 5.2.10.239 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax](#rqsrs-006rbacsettingsprofilealtersyntax) - * 5.2.10.240 [RQ.SRS-006.RBAC.SettingsProfile.Drop](#rqsrs-006rbacsettingsprofiledrop) - * 5.2.10.241 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Effect](#rqsrs-006rbacsettingsprofiledropeffect) - * 5.2.10.242 [RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists](#rqsrs-006rbacsettingsprofiledropifexists) - * 5.2.10.243 [RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster](#rqsrs-006rbacsettingsprofiledroponcluster) - * 5.2.10.244 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax](#rqsrs-006rbacsettingsprofiledropsyntax) - * 5.2.10.245 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile](#rqsrs-006rbacsettingsprofileshowcreatesettingsprofile) - * 5.2.10.246 [RQ.SRS-006.RBAC.Quota.Create](#rqsrs-006rbacquotacreate) - * 5.2.10.247 [RQ.SRS-006.RBAC.Quota.Create.Effect](#rqsrs-006rbacquotacreateeffect) - * 5.2.10.248 [RQ.SRS-006.RBAC.Quota.Create.IfNotExists](#rqsrs-006rbacquotacreateifnotexists) - * 5.2.10.249 [RQ.SRS-006.RBAC.Quota.Create.Replace](#rqsrs-006rbacquotacreatereplace) - * 5.2.10.250 [RQ.SRS-006.RBAC.Quota.Create.Cluster](#rqsrs-006rbacquotacreatecluster) - * 5.2.10.251 [RQ.SRS-006.RBAC.Quota.Create.Interval](#rqsrs-006rbacquotacreateinterval) - * 5.2.10.252 [RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized](#rqsrs-006rbacquotacreateintervalrandomized) - * 5.2.10.253 [RQ.SRS-006.RBAC.Quota.Create.Queries](#rqsrs-006rbacquotacreatequeries) - * 5.2.10.254 [RQ.SRS-006.RBAC.Quota.Create.Errors](#rqsrs-006rbacquotacreateerrors) - * 5.2.10.255 [RQ.SRS-006.RBAC.Quota.Create.ResultRows](#rqsrs-006rbacquotacreateresultrows) - * 5.2.10.256 [RQ.SRS-006.RBAC.Quota.Create.ReadRows](#rqsrs-006rbacquotacreatereadrows) - * 5.2.10.257 [RQ.SRS-006.RBAC.Quota.Create.ResultBytes](#rqsrs-006rbacquotacreateresultbytes) - * 5.2.10.258 [RQ.SRS-006.RBAC.Quota.Create.ReadBytes](#rqsrs-006rbacquotacreatereadbytes) - * 5.2.10.259 [RQ.SRS-006.RBAC.Quota.Create.ExecutionTime](#rqsrs-006rbacquotacreateexecutiontime) - * 5.2.10.260 [RQ.SRS-006.RBAC.Quota.Create.NoLimits](#rqsrs-006rbacquotacreatenolimits) - * 5.2.10.261 [RQ.SRS-006.RBAC.Quota.Create.TrackingOnly](#rqsrs-006rbacquotacreatetrackingonly) - * 5.2.10.262 [RQ.SRS-006.RBAC.Quota.Create.KeyedBy](#rqsrs-006rbacquotacreatekeyedby) - * 5.2.10.263 [RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions](#rqsrs-006rbacquotacreatekeyedbyoptions) - * 5.2.10.264 [RQ.SRS-006.RBAC.Quota.Create.Assignment](#rqsrs-006rbacquotacreateassignment) - * 5.2.10.265 [RQ.SRS-006.RBAC.Quota.Create.Assignment.None](#rqsrs-006rbacquotacreateassignmentnone) - * 5.2.10.266 [RQ.SRS-006.RBAC.Quota.Create.Assignment.All](#rqsrs-006rbacquotacreateassignmentall) - * 5.2.10.267 [RQ.SRS-006.RBAC.Quota.Create.Assignment.Except](#rqsrs-006rbacquotacreateassignmentexcept) - * 5.2.10.268 [RQ.SRS-006.RBAC.Quota.Create.Syntax](#rqsrs-006rbacquotacreatesyntax) - * 5.2.10.269 [RQ.SRS-006.RBAC.Quota.Alter](#rqsrs-006rbacquotaalter) - * 5.2.10.270 [RQ.SRS-006.RBAC.Quota.Alter.Effect](#rqsrs-006rbacquotaaltereffect) - * 5.2.10.271 [RQ.SRS-006.RBAC.Quota.Alter.IfExists](#rqsrs-006rbacquotaalterifexists) - * 5.2.10.272 [RQ.SRS-006.RBAC.Quota.Alter.Rename](#rqsrs-006rbacquotaalterrename) - * 5.2.10.273 [RQ.SRS-006.RBAC.Quota.Alter.Cluster](#rqsrs-006rbacquotaaltercluster) - * 5.2.10.274 [RQ.SRS-006.RBAC.Quota.Alter.Interval](#rqsrs-006rbacquotaalterinterval) - * 5.2.10.275 [RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized](#rqsrs-006rbacquotaalterintervalrandomized) - * 5.2.10.276 [RQ.SRS-006.RBAC.Quota.Alter.Queries](#rqsrs-006rbacquotaalterqueries) - * 5.2.10.277 [RQ.SRS-006.RBAC.Quota.Alter.Errors](#rqsrs-006rbacquotaaltererrors) - * 5.2.10.278 [RQ.SRS-006.RBAC.Quota.Alter.ResultRows](#rqsrs-006rbacquotaalterresultrows) - * 5.2.10.279 [RQ.SRS-006.RBAC.Quota.Alter.ReadRows](#rqsrs-006rbacquotaalterreadrows) - * 5.2.10.280 [RQ.SRS-006.RBAC.Quota.ALter.ResultBytes](#rqsrs-006rbacquotaalterresultbytes) - * 5.2.10.281 [RQ.SRS-006.RBAC.Quota.Alter.ReadBytes](#rqsrs-006rbacquotaalterreadbytes) - * 5.2.10.282 [RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime](#rqsrs-006rbacquotaalterexecutiontime) - * 5.2.10.283 [RQ.SRS-006.RBAC.Quota.Alter.NoLimits](#rqsrs-006rbacquotaalternolimits) - * 5.2.10.284 [RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly](#rqsrs-006rbacquotaaltertrackingonly) - * 5.2.10.285 [RQ.SRS-006.RBAC.Quota.Alter.KeyedBy](#rqsrs-006rbacquotaalterkeyedby) - * 5.2.10.286 [RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions](#rqsrs-006rbacquotaalterkeyedbyoptions) - * 5.2.10.287 [RQ.SRS-006.RBAC.Quota.Alter.Assignment](#rqsrs-006rbacquotaalterassignment) - * 5.2.10.288 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.None](#rqsrs-006rbacquotaalterassignmentnone) - * 5.2.10.289 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.All](#rqsrs-006rbacquotaalterassignmentall) - * 5.2.10.290 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except](#rqsrs-006rbacquotaalterassignmentexcept) - * 5.2.10.291 [RQ.SRS-006.RBAC.Quota.Alter.Syntax](#rqsrs-006rbacquotaaltersyntax) - * 5.2.10.292 [RQ.SRS-006.RBAC.Quota.Drop](#rqsrs-006rbacquotadrop) - * 5.2.10.293 [RQ.SRS-006.RBAC.Quota.Drop.Effect](#rqsrs-006rbacquotadropeffect) - * 5.2.10.294 [RQ.SRS-006.RBAC.Quota.Drop.IfExists](#rqsrs-006rbacquotadropifexists) - * 5.2.10.295 [RQ.SRS-006.RBAC.Quota.Drop.Cluster](#rqsrs-006rbacquotadropcluster) - * 5.2.10.296 [RQ.SRS-006.RBAC.Quota.Drop.Syntax](#rqsrs-006rbacquotadropsyntax) - * 5.2.10.297 [RQ.SRS-006.RBAC.Quota.ShowQuotas](#rqsrs-006rbacquotashowquotas) - * 5.2.10.298 [RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile](#rqsrs-006rbacquotashowquotasintooutfile) - * 5.2.10.299 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Format](#rqsrs-006rbacquotashowquotasformat) - * 5.2.10.300 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings](#rqsrs-006rbacquotashowquotassettings) - * 5.2.10.301 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax](#rqsrs-006rbacquotashowquotassyntax) - * 5.2.10.302 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name](#rqsrs-006rbacquotashowcreatequotaname) - * 5.2.10.303 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current](#rqsrs-006rbacquotashowcreatequotacurrent) - * 5.2.10.304 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax](#rqsrs-006rbacquotashowcreatequotasyntax) - * 5.2.10.305 [RQ.SRS-006.RBAC.RowPolicy.Create](#rqsrs-006rbacrowpolicycreate) - * 5.2.10.306 [RQ.SRS-006.RBAC.RowPolicy.Create.Effect](#rqsrs-006rbacrowpolicycreateeffect) - * 5.2.10.307 [RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists](#rqsrs-006rbacrowpolicycreateifnotexists) - * 5.2.10.308 [RQ.SRS-006.RBAC.RowPolicy.Create.Replace](#rqsrs-006rbacrowpolicycreatereplace) - * 5.2.10.309 [RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster](#rqsrs-006rbacrowpolicycreateoncluster) - * 5.2.10.310 [RQ.SRS-006.RBAC.RowPolicy.Create.On](#rqsrs-006rbacrowpolicycreateon) - * 5.2.10.311 [RQ.SRS-006.RBAC.RowPolicy.Create.Access](#rqsrs-006rbacrowpolicycreateaccess) - * 5.2.10.312 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive](#rqsrs-006rbacrowpolicycreateaccesspermissive) - * 5.2.10.313 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive](#rqsrs-006rbacrowpolicycreateaccessrestrictive) - * 5.2.10.314 [RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect](#rqsrs-006rbacrowpolicycreateforselect) - * 5.2.10.315 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition](#rqsrs-006rbacrowpolicycreatecondition) - * 5.2.10.316 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition.Effect](#rqsrs-006rbacrowpolicycreateconditioneffect) - * 5.2.10.317 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment](#rqsrs-006rbacrowpolicycreateassignment) - * 5.2.10.318 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None](#rqsrs-006rbacrowpolicycreateassignmentnone) - * 5.2.10.319 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All](#rqsrs-006rbacrowpolicycreateassignmentall) - * 5.2.10.320 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept](#rqsrs-006rbacrowpolicycreateassignmentallexcept) - * 5.2.10.321 [RQ.SRS-006.RBAC.RowPolicy.Create.Syntax](#rqsrs-006rbacrowpolicycreatesyntax) - * 5.2.10.322 [RQ.SRS-006.RBAC.RowPolicy.Alter](#rqsrs-006rbacrowpolicyalter) - * 5.2.10.323 [RQ.SRS-006.RBAC.RowPolicy.Alter.Effect](#rqsrs-006rbacrowpolicyaltereffect) - * 5.2.10.324 [RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists](#rqsrs-006rbacrowpolicyalterifexists) - * 5.2.10.325 [RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect](#rqsrs-006rbacrowpolicyalterforselect) - * 5.2.10.326 [RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster](#rqsrs-006rbacrowpolicyalteroncluster) - * 5.2.10.327 [RQ.SRS-006.RBAC.RowPolicy.Alter.On](#rqsrs-006rbacrowpolicyalteron) - * 5.2.10.328 [RQ.SRS-006.RBAC.RowPolicy.Alter.Rename](#rqsrs-006rbacrowpolicyalterrename) - * 5.2.10.329 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access](#rqsrs-006rbacrowpolicyalteraccess) - * 5.2.10.330 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive](#rqsrs-006rbacrowpolicyalteraccesspermissive) - * 5.2.10.331 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive](#rqsrs-006rbacrowpolicyalteraccessrestrictive) - * 5.2.10.332 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition](#rqsrs-006rbacrowpolicyaltercondition) - * 5.2.10.333 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.Effect](#rqsrs-006rbacrowpolicyalterconditioneffect) - * 5.2.10.334 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None](#rqsrs-006rbacrowpolicyalterconditionnone) - * 5.2.10.335 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment](#rqsrs-006rbacrowpolicyalterassignment) - * 5.2.10.336 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None](#rqsrs-006rbacrowpolicyalterassignmentnone) - * 5.2.10.337 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All](#rqsrs-006rbacrowpolicyalterassignmentall) - * 5.2.10.338 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept](#rqsrs-006rbacrowpolicyalterassignmentallexcept) - * 5.2.10.339 [RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax](#rqsrs-006rbacrowpolicyaltersyntax) - * 5.2.10.340 [RQ.SRS-006.RBAC.RowPolicy.Drop](#rqsrs-006rbacrowpolicydrop) - * 5.2.10.341 [RQ.SRS-006.RBAC.RowPolicy.Drop.Effect](#rqsrs-006rbacrowpolicydropeffect) - * 5.2.10.342 [RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists](#rqsrs-006rbacrowpolicydropifexists) - * 5.2.10.343 [RQ.SRS-006.RBAC.RowPolicy.Drop.On](#rqsrs-006rbacrowpolicydropon) - * 5.2.10.344 [RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster](#rqsrs-006rbacrowpolicydroponcluster) - * 5.2.10.345 [RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax](#rqsrs-006rbacrowpolicydropsyntax) - * 5.2.10.346 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy](#rqsrs-006rbacrowpolicyshowcreaterowpolicy) - * 5.2.10.347 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On](#rqsrs-006rbacrowpolicyshowcreaterowpolicyon) - * 5.2.10.348 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax](#rqsrs-006rbacrowpolicyshowcreaterowpolicysyntax) - * 5.2.10.349 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies](#rqsrs-006rbacrowpolicyshowrowpolicies) - * 5.2.10.350 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On](#rqsrs-006rbacrowpolicyshowrowpolicieson) - * 5.2.10.351 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) + * 5.2.8.1 [RQ.SRS-006.RBAC.User.Use.DefaultRole](#rqsrs-006rbacuserusedefaultrole) + * 5.2.8.2 [RQ.SRS-006.RBAC.User.Use.AllRolesWhenNoDefaultRole](#rqsrs-006rbacuseruseallroleswhennodefaultrole) + * 5.2.8.3 [RQ.SRS-006.RBAC.User.Create](#rqsrs-006rbacusercreate) + * 5.2.8.4 [RQ.SRS-006.RBAC.User.Create.IfNotExists](#rqsrs-006rbacusercreateifnotexists) + * 5.2.8.5 [RQ.SRS-006.RBAC.User.Create.Replace](#rqsrs-006rbacusercreatereplace) + * 5.2.8.6 [RQ.SRS-006.RBAC.User.Create.Password.NoPassword](#rqsrs-006rbacusercreatepasswordnopassword) + * 5.2.8.7 [RQ.SRS-006.RBAC.User.Create.Password.NoPassword.Login](#rqsrs-006rbacusercreatepasswordnopasswordlogin) + * 5.2.8.8 [RQ.SRS-006.RBAC.User.Create.Password.PlainText](#rqsrs-006rbacusercreatepasswordplaintext) + * 5.2.8.9 [RQ.SRS-006.RBAC.User.Create.Password.PlainText.Login](#rqsrs-006rbacusercreatepasswordplaintextlogin) + * 5.2.8.10 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Password](#rqsrs-006rbacusercreatepasswordsha256password) + * 5.2.8.11 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Password.Login](#rqsrs-006rbacusercreatepasswordsha256passwordlogin) + * 5.2.8.12 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash](#rqsrs-006rbacusercreatepasswordsha256hash) + * 5.2.8.13 [RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash.Login](#rqsrs-006rbacusercreatepasswordsha256hashlogin) + * 5.2.8.14 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password](#rqsrs-006rbacusercreatepassworddoublesha1password) + * 5.2.8.15 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password.Login](#rqsrs-006rbacusercreatepassworddoublesha1passwordlogin) + * 5.2.8.16 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash](#rqsrs-006rbacusercreatepassworddoublesha1hash) + * 5.2.8.17 [RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash.Login](#rqsrs-006rbacusercreatepassworddoublesha1hashlogin) + * 5.2.8.18 [RQ.SRS-006.RBAC.User.Create.Host.Name](#rqsrs-006rbacusercreatehostname) + * 5.2.8.19 [RQ.SRS-006.RBAC.User.Create.Host.Regexp](#rqsrs-006rbacusercreatehostregexp) + * 5.2.8.20 [RQ.SRS-006.RBAC.User.Create.Host.IP](#rqsrs-006rbacusercreatehostip) + * 5.2.8.21 [RQ.SRS-006.RBAC.User.Create.Host.Any](#rqsrs-006rbacusercreatehostany) + * 5.2.8.22 [RQ.SRS-006.RBAC.User.Create.Host.None](#rqsrs-006rbacusercreatehostnone) + * 5.2.8.23 [RQ.SRS-006.RBAC.User.Create.Host.Local](#rqsrs-006rbacusercreatehostlocal) + * 5.2.8.24 [RQ.SRS-006.RBAC.User.Create.Host.Like](#rqsrs-006rbacusercreatehostlike) + * 5.2.8.25 [RQ.SRS-006.RBAC.User.Create.Host.Default](#rqsrs-006rbacusercreatehostdefault) + * 5.2.8.26 [RQ.SRS-006.RBAC.User.Create.DefaultRole](#rqsrs-006rbacusercreatedefaultrole) + * 5.2.8.27 [RQ.SRS-006.RBAC.User.Create.DefaultRole.None](#rqsrs-006rbacusercreatedefaultrolenone) + * 5.2.8.28 [RQ.SRS-006.RBAC.User.Create.DefaultRole.All](#rqsrs-006rbacusercreatedefaultroleall) + * 5.2.8.29 [RQ.SRS-006.RBAC.User.Create.Settings](#rqsrs-006rbacusercreatesettings) + * 5.2.8.30 [RQ.SRS-006.RBAC.User.Create.OnCluster](#rqsrs-006rbacusercreateoncluster) + * 5.2.8.31 [RQ.SRS-006.RBAC.User.Create.Syntax](#rqsrs-006rbacusercreatesyntax) + * 5.2.8.32 [RQ.SRS-006.RBAC.User.Alter](#rqsrs-006rbacuseralter) + * 5.2.8.33 [RQ.SRS-006.RBAC.User.Alter.OrderOfEvaluation](#rqsrs-006rbacuseralterorderofevaluation) + * 5.2.8.34 [RQ.SRS-006.RBAC.User.Alter.IfExists](#rqsrs-006rbacuseralterifexists) + * 5.2.8.35 [RQ.SRS-006.RBAC.User.Alter.Cluster](#rqsrs-006rbacuseraltercluster) + * 5.2.8.36 [RQ.SRS-006.RBAC.User.Alter.Rename](#rqsrs-006rbacuseralterrename) + * 5.2.8.37 [RQ.SRS-006.RBAC.User.Alter.Password.PlainText](#rqsrs-006rbacuseralterpasswordplaintext) + * 5.2.8.38 [RQ.SRS-006.RBAC.User.Alter.Password.Sha256Password](#rqsrs-006rbacuseralterpasswordsha256password) + * 5.2.8.39 [RQ.SRS-006.RBAC.User.Alter.Password.DoubleSha1Password](#rqsrs-006rbacuseralterpassworddoublesha1password) + * 5.2.8.40 [RQ.SRS-006.RBAC.User.Alter.Host.AddDrop](#rqsrs-006rbacuseralterhostadddrop) + * 5.2.8.41 [RQ.SRS-006.RBAC.User.Alter.Host.Local](#rqsrs-006rbacuseralterhostlocal) + * 5.2.8.42 [RQ.SRS-006.RBAC.User.Alter.Host.Name](#rqsrs-006rbacuseralterhostname) + * 5.2.8.43 [RQ.SRS-006.RBAC.User.Alter.Host.Regexp](#rqsrs-006rbacuseralterhostregexp) + * 5.2.8.44 [RQ.SRS-006.RBAC.User.Alter.Host.IP](#rqsrs-006rbacuseralterhostip) + * 5.2.8.45 [RQ.SRS-006.RBAC.User.Alter.Host.Like](#rqsrs-006rbacuseralterhostlike) + * 5.2.8.46 [RQ.SRS-006.RBAC.User.Alter.Host.Any](#rqsrs-006rbacuseralterhostany) + * 5.2.8.47 [RQ.SRS-006.RBAC.User.Alter.Host.None](#rqsrs-006rbacuseralterhostnone) + * 5.2.8.48 [RQ.SRS-006.RBAC.User.Alter.DefaultRole](#rqsrs-006rbacuseralterdefaultrole) + * 5.2.8.49 [RQ.SRS-006.RBAC.User.Alter.DefaultRole.All](#rqsrs-006rbacuseralterdefaultroleall) + * 5.2.8.50 [RQ.SRS-006.RBAC.User.Alter.DefaultRole.AllExcept](#rqsrs-006rbacuseralterdefaultroleallexcept) + * 5.2.8.51 [RQ.SRS-006.RBAC.User.Alter.Settings](#rqsrs-006rbacuseraltersettings) + * 5.2.8.52 [RQ.SRS-006.RBAC.User.Alter.Settings.Min](#rqsrs-006rbacuseraltersettingsmin) + * 5.2.8.53 [RQ.SRS-006.RBAC.User.Alter.Settings.Max](#rqsrs-006rbacuseraltersettingsmax) + * 5.2.8.54 [RQ.SRS-006.RBAC.User.Alter.Settings.Profile](#rqsrs-006rbacuseraltersettingsprofile) + * 5.2.8.55 [RQ.SRS-006.RBAC.User.Alter.Syntax](#rqsrs-006rbacuseraltersyntax) + * 5.2.8.56 [RQ.SRS-006.RBAC.SetDefaultRole](#rqsrs-006rbacsetdefaultrole) + * 5.2.8.57 [RQ.SRS-006.RBAC.SetDefaultRole.CurrentUser](#rqsrs-006rbacsetdefaultrolecurrentuser) + * 5.2.8.58 [RQ.SRS-006.RBAC.SetDefaultRole.All](#rqsrs-006rbacsetdefaultroleall) + * 5.2.8.59 [RQ.SRS-006.RBAC.SetDefaultRole.AllExcept](#rqsrs-006rbacsetdefaultroleallexcept) + * 5.2.8.60 [RQ.SRS-006.RBAC.SetDefaultRole.None](#rqsrs-006rbacsetdefaultrolenone) + * 5.2.8.61 [RQ.SRS-006.RBAC.SetDefaultRole.Syntax](#rqsrs-006rbacsetdefaultrolesyntax) + * 5.2.8.62 [RQ.SRS-006.RBAC.SetRole](#rqsrs-006rbacsetrole) + * 5.2.8.63 [RQ.SRS-006.RBAC.SetRole.Default](#rqsrs-006rbacsetroledefault) + * 5.2.8.64 [RQ.SRS-006.RBAC.SetRole.None](#rqsrs-006rbacsetrolenone) + * 5.2.8.65 [RQ.SRS-006.RBAC.SetRole.All](#rqsrs-006rbacsetroleall) + * 5.2.8.66 [RQ.SRS-006.RBAC.SetRole.AllExcept](#rqsrs-006rbacsetroleallexcept) + * 5.2.8.67 [RQ.SRS-006.RBAC.SetRole.Syntax](#rqsrs-006rbacsetrolesyntax) + * 5.2.8.68 [RQ.SRS-006.RBAC.User.ShowCreateUser](#rqsrs-006rbacusershowcreateuser) + * 5.2.8.69 [RQ.SRS-006.RBAC.User.ShowCreateUser.For](#rqsrs-006rbacusershowcreateuserfor) + * 5.2.8.70 [RQ.SRS-006.RBAC.User.ShowCreateUser.Syntax](#rqsrs-006rbacusershowcreateusersyntax) + * 5.2.8.71 [RQ.SRS-006.RBAC.User.Drop](#rqsrs-006rbacuserdrop) + * 5.2.8.72 [RQ.SRS-006.RBAC.User.Drop.IfExists](#rqsrs-006rbacuserdropifexists) + * 5.2.8.73 [RQ.SRS-006.RBAC.User.Drop.OnCluster](#rqsrs-006rbacuserdroponcluster) + * 5.2.8.74 [RQ.SRS-006.RBAC.User.Drop.Syntax](#rqsrs-006rbacuserdropsyntax) + * 5.2.8.75 [RQ.SRS-006.RBAC.Role.Create](#rqsrs-006rbacrolecreate) + * 5.2.8.76 [RQ.SRS-006.RBAC.Role.Create.IfNotExists](#rqsrs-006rbacrolecreateifnotexists) + * 5.2.8.77 [RQ.SRS-006.RBAC.Role.Create.Replace](#rqsrs-006rbacrolecreatereplace) + * 5.2.8.78 [RQ.SRS-006.RBAC.Role.Create.Settings](#rqsrs-006rbacrolecreatesettings) + * 5.2.8.79 [RQ.SRS-006.RBAC.Role.Create.Syntax](#rqsrs-006rbacrolecreatesyntax) + * 5.2.8.80 [RQ.SRS-006.RBAC.Role.Create.Effect](#rqsrs-006rbacrolecreateeffect) + * 5.2.8.81 [RQ.SRS-006.RBAC.Role.Alter](#rqsrs-006rbacrolealter) + * 5.2.8.82 [RQ.SRS-006.RBAC.Role.Alter.IfExists](#rqsrs-006rbacrolealterifexists) + * 5.2.8.83 [RQ.SRS-006.RBAC.Role.Alter.Cluster](#rqsrs-006rbacrolealtercluster) + * 5.2.8.84 [RQ.SRS-006.RBAC.Role.Alter.Rename](#rqsrs-006rbacrolealterrename) + * 5.2.8.85 [RQ.SRS-006.RBAC.Role.Alter.Settings](#rqsrs-006rbacrolealtersettings) + * 5.2.8.86 [RQ.SRS-006.RBAC.Role.Alter.Effect](#rqsrs-006rbacrolealtereffect) + * 5.2.8.87 [RQ.SRS-006.RBAC.Role.Alter.Syntax](#rqsrs-006rbacrolealtersyntax) + * 5.2.8.88 [RQ.SRS-006.RBAC.Role.Drop](#rqsrs-006rbacroledrop) + * 5.2.8.89 [RQ.SRS-006.RBAC.Role.Drop.IfExists](#rqsrs-006rbacroledropifexists) + * 5.2.8.90 [RQ.SRS-006.RBAC.Role.Drop.Cluster](#rqsrs-006rbacroledropcluster) + * 5.2.8.91 [RQ.SRS-006.RBAC.Role.Drop.Effect](#rqsrs-006rbacroledropeffect) + * 5.2.8.92 [RQ.SRS-006.RBAC.Role.Drop.Syntax](#rqsrs-006rbacroledropsyntax) + * 5.2.8.93 [RQ.SRS-006.RBAC.Role.ShowCreate](#rqsrs-006rbacroleshowcreate) + * 5.2.8.94 [RQ.SRS-006.RBAC.Role.ShowCreate.Syntax](#rqsrs-006rbacroleshowcreatesyntax) + * 5.2.8.95 [RQ.SRS-006.RBAC.Grant.Privilege.To](#rqsrs-006rbacgrantprivilegeto) + * 5.2.8.96 [RQ.SRS-006.RBAC.Grant.Privilege.To.Effect](#rqsrs-006rbacgrantprivilegetoeffect) + * 5.2.8.97 [RQ.SRS-006.RBAC.Grant.Privilege.ToCurrentUser](#rqsrs-006rbacgrantprivilegetocurrentuser) + * 5.2.8.98 [RQ.SRS-006.RBAC.Grant.Privilege.Select](#rqsrs-006rbacgrantprivilegeselect) + * 5.2.8.99 [RQ.SRS-006.RBAC.Grant.Privilege.Insert](#rqsrs-006rbacgrantprivilegeinsert) + * 5.2.8.100 [RQ.SRS-006.RBAC.Grant.Privilege.Alter](#rqsrs-006rbacgrantprivilegealter) + * 5.2.8.101 [RQ.SRS-006.RBAC.Grant.Privilege.Alter.Effect](#rqsrs-006rbacgrantprivilegealtereffect) + * 5.2.8.102 [RQ.SRS-006.RBAC.Grant.Privilege.Create](#rqsrs-006rbacgrantprivilegecreate) + * 5.2.8.103 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Effect](#rqsrs-006rbacgrantprivilegecreateeffect) + * 5.2.8.104 [RQ.SRS-006.RBAC.Grant.Privilege.Drop](#rqsrs-006rbacgrantprivilegedrop) + * 5.2.8.105 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Effect](#rqsrs-006rbacgrantprivilegedropeffect) + * 5.2.8.106 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate](#rqsrs-006rbacgrantprivilegetruncate) + * 5.2.8.107 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate.Effect](#rqsrs-006rbacgrantprivilegetruncateeffect) + * 5.2.8.108 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize](#rqsrs-006rbacgrantprivilegeoptimize) + * 5.2.8.109 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize.Effect](#rqsrs-006rbacgrantprivilegeoptimizeeffect) + * 5.2.8.110 [RQ.SRS-006.RBAC.Grant.Privilege.Show](#rqsrs-006rbacgrantprivilegeshow) + * 5.2.8.111 [RQ.SRS-006.RBAC.Grant.Privilege.Show.Effect](#rqsrs-006rbacgrantprivilegeshoweffect) + * 5.2.8.112 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery](#rqsrs-006rbacgrantprivilegekillquery) + * 5.2.8.113 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery.Effect](#rqsrs-006rbacgrantprivilegekillqueryeffect) + * 5.2.8.114 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement](#rqsrs-006rbacgrantprivilegeaccessmanagement) + * 5.2.8.115 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement.Effect](#rqsrs-006rbacgrantprivilegeaccessmanagementeffect) + * 5.2.8.116 [RQ.SRS-006.RBAC.Grant.Privilege.System](#rqsrs-006rbacgrantprivilegesystem) + * 5.2.8.117 [RQ.SRS-006.RBAC.Grant.Privilege.System.Effect](#rqsrs-006rbacgrantprivilegesystemeffect) + * 5.2.8.118 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection](#rqsrs-006rbacgrantprivilegeintrospection) + * 5.2.8.119 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection.Effect](#rqsrs-006rbacgrantprivilegeintrospectioneffect) + * 5.2.8.120 [RQ.SRS-006.RBAC.Grant.Privilege.Sources](#rqsrs-006rbacgrantprivilegesources) + * 5.2.8.121 [RQ.SRS-006.RBAC.Grant.Privilege.Sources.Effect](#rqsrs-006rbacgrantprivilegesourceseffect) + * 5.2.8.122 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet](#rqsrs-006rbacgrantprivilegedictget) + * 5.2.8.123 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet.Effect](#rqsrs-006rbacgrantprivilegedictgeteffect) + * 5.2.8.124 [RQ.SRS-006.RBAC.Grant.Privilege.None](#rqsrs-006rbacgrantprivilegenone) + * 5.2.8.125 [RQ.SRS-006.RBAC.Grant.Privilege.None.Effect](#rqsrs-006rbacgrantprivilegenoneeffect) + * 5.2.8.126 [RQ.SRS-006.RBAC.Grant.Privilege.All](#rqsrs-006rbacgrantprivilegeall) + * 5.2.8.127 [RQ.SRS-006.RBAC.Grant.Privilege.All.Effect](#rqsrs-006rbacgrantprivilegealleffect) + * 5.2.8.128 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption](#rqsrs-006rbacgrantprivilegegrantoption) + * 5.2.8.129 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption.Effect](#rqsrs-006rbacgrantprivilegegrantoptioneffect) + * 5.2.8.130 [RQ.SRS-006.RBAC.Grant.Privilege.On](#rqsrs-006rbacgrantprivilegeon) + * 5.2.8.131 [RQ.SRS-006.RBAC.Grant.Privilege.On.Effect](#rqsrs-006rbacgrantprivilegeoneffect) + * 5.2.8.132 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns](#rqsrs-006rbacgrantprivilegeprivilegecolumns) + * 5.2.8.133 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns.Effect](#rqsrs-006rbacgrantprivilegeprivilegecolumnseffect) + * 5.2.8.134 [RQ.SRS-006.RBAC.Grant.Privilege.OnCluster](#rqsrs-006rbacgrantprivilegeoncluster) + * 5.2.8.135 [RQ.SRS-006.RBAC.Grant.Privilege.Syntax](#rqsrs-006rbacgrantprivilegesyntax) + * 5.2.8.136 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster](#rqsrs-006rbacrevokeprivilegecluster) + * 5.2.8.137 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster.Effect](#rqsrs-006rbacrevokeprivilegeclustereffect) + * 5.2.8.138 [RQ.SRS-006.RBAC.Revoke.Privilege.Any](#rqsrs-006rbacrevokeprivilegeany) + * 5.2.8.139 [RQ.SRS-006.RBAC.Revoke.Privilege.Any.Effect](#rqsrs-006rbacrevokeprivilegeanyeffect) + * 5.2.8.140 [RQ.SRS-006.RBAC.Revoke.Privilege.Select](#rqsrs-006rbacrevokeprivilegeselect) + * 5.2.8.141 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert](#rqsrs-006rbacrevokeprivilegeinsert) + * 5.2.8.142 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter](#rqsrs-006rbacrevokeprivilegealter) + * 5.2.8.143 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter.Effect](#rqsrs-006rbacrevokeprivilegealtereffect) + * 5.2.8.144 [RQ.SRS-006.RBAC.Revoke.Privilege.Create](#rqsrs-006rbacrevokeprivilegecreate) + * 5.2.8.145 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Effect](#rqsrs-006rbacrevokeprivilegecreateeffect) + * 5.2.8.146 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop](#rqsrs-006rbacrevokeprivilegedrop) + * 5.2.8.147 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Effect](#rqsrs-006rbacrevokeprivilegedropeffect) + * 5.2.8.148 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate](#rqsrs-006rbacrevokeprivilegetruncate) + * 5.2.8.149 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate.Effect](#rqsrs-006rbacrevokeprivilegetruncateeffect) + * 5.2.8.150 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize](#rqsrs-006rbacrevokeprivilegeoptimize) + * 5.2.8.151 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize.Effect](#rqsrs-006rbacrevokeprivilegeoptimizeeffect) + * 5.2.8.152 [RQ.SRS-006.RBAC.Revoke.Privilege.Show](#rqsrs-006rbacrevokeprivilegeshow) + * 5.2.8.153 [RQ.SRS-006.RBAC.Revoke.Privilege.Show.Effect](#rqsrs-006rbacrevokeprivilegeshoweffect) + * 5.2.8.154 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery](#rqsrs-006rbacrevokeprivilegekillquery) + * 5.2.8.155 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery.Effect](#rqsrs-006rbacrevokeprivilegekillqueryeffect) + * 5.2.8.156 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement](#rqsrs-006rbacrevokeprivilegeaccessmanagement) + * 5.2.8.157 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement.Effect](#rqsrs-006rbacrevokeprivilegeaccessmanagementeffect) + * 5.2.8.158 [RQ.SRS-006.RBAC.Revoke.Privilege.System](#rqsrs-006rbacrevokeprivilegesystem) + * 5.2.8.159 [RQ.SRS-006.RBAC.Revoke.Privilege.System.Effect](#rqsrs-006rbacrevokeprivilegesystemeffect) + * 5.2.8.160 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection](#rqsrs-006rbacrevokeprivilegeintrospection) + * 5.2.8.161 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection.Effect](#rqsrs-006rbacrevokeprivilegeintrospectioneffect) + * 5.2.8.162 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources](#rqsrs-006rbacrevokeprivilegesources) + * 5.2.8.163 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources.Effect](#rqsrs-006rbacrevokeprivilegesourceseffect) + * 5.2.8.164 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet](#rqsrs-006rbacrevokeprivilegedictget) + * 5.2.8.165 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet.Effect](#rqsrs-006rbacrevokeprivilegedictgeteffect) + * 5.2.8.166 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns](#rqsrs-006rbacrevokeprivilegeprivelegecolumns) + * 5.2.8.167 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns.Effect](#rqsrs-006rbacrevokeprivilegeprivelegecolumnseffect) + * 5.2.8.168 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple](#rqsrs-006rbacrevokeprivilegemultiple) + * 5.2.8.169 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple.Effect](#rqsrs-006rbacrevokeprivilegemultipleeffect) + * 5.2.8.170 [RQ.SRS-006.RBAC.Revoke.Privilege.All](#rqsrs-006rbacrevokeprivilegeall) + * 5.2.8.171 [RQ.SRS-006.RBAC.Revoke.Privilege.All.Effect](#rqsrs-006rbacrevokeprivilegealleffect) + * 5.2.8.172 [RQ.SRS-006.RBAC.Revoke.Privilege.None](#rqsrs-006rbacrevokeprivilegenone) + * 5.2.8.173 [RQ.SRS-006.RBAC.Revoke.Privilege.None.Effect](#rqsrs-006rbacrevokeprivilegenoneeffect) + * 5.2.8.174 [RQ.SRS-006.RBAC.Revoke.Privilege.On](#rqsrs-006rbacrevokeprivilegeon) + * 5.2.8.175 [RQ.SRS-006.RBAC.Revoke.Privilege.On.Effect](#rqsrs-006rbacrevokeprivilegeoneffect) + * 5.2.8.176 [RQ.SRS-006.RBAC.Revoke.Privilege.From](#rqsrs-006rbacrevokeprivilegefrom) + * 5.2.8.177 [RQ.SRS-006.RBAC.Revoke.Privilege.From.Effect](#rqsrs-006rbacrevokeprivilegefromeffect) + * 5.2.8.178 [RQ.SRS-006.RBAC.Revoke.Privilege.Syntax](#rqsrs-006rbacrevokeprivilegesyntax) + * 5.2.8.179 [RQ.SRS-006.RBAC.PartialRevoke.Syntax](#rqsrs-006rbacpartialrevokesyntax) + * 5.2.8.180 [RQ.SRS-006.RBAC.PartialRevoke.Effect](#rqsrs-006rbacpartialrevokeeffect) + * 5.2.8.181 [RQ.SRS-006.RBAC.Grant.Role](#rqsrs-006rbacgrantrole) + * 5.2.8.182 [RQ.SRS-006.RBAC.Grant.Role.Effect](#rqsrs-006rbacgrantroleeffect) + * 5.2.8.183 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser](#rqsrs-006rbacgrantrolecurrentuser) + * 5.2.8.184 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser.Effect](#rqsrs-006rbacgrantrolecurrentusereffect) + * 5.2.8.185 [RQ.SRS-006.RBAC.Grant.Role.AdminOption](#rqsrs-006rbacgrantroleadminoption) + * 5.2.8.186 [RQ.SRS-006.RBAC.Grant.Role.AdminOption.Effect](#rqsrs-006rbacgrantroleadminoptioneffect) + * 5.2.8.187 [RQ.SRS-006.RBAC.Grant.Role.OnCluster](#rqsrs-006rbacgrantroleoncluster) + * 5.2.8.188 [RQ.SRS-006.RBAC.Grant.Role.Syntax](#rqsrs-006rbacgrantrolesyntax) + * 5.2.8.189 [RQ.SRS-006.RBAC.Revoke.Role](#rqsrs-006rbacrevokerole) + * 5.2.8.190 [RQ.SRS-006.RBAC.Revoke.Role.Effect](#rqsrs-006rbacrevokeroleeffect) + * 5.2.8.191 [RQ.SRS-006.RBAC.Revoke.Role.Keywords](#rqsrs-006rbacrevokerolekeywords) + * 5.2.8.192 [RQ.SRS-006.RBAC.Revoke.Role.Keywords.Effect](#rqsrs-006rbacrevokerolekeywordseffect) + * 5.2.8.193 [RQ.SRS-006.RBAC.Revoke.Role.Cluster](#rqsrs-006rbacrevokerolecluster) + * 5.2.8.194 [RQ.SRS-006.RBAC.Revoke.Role.Cluster.Effect](#rqsrs-006rbacrevokeroleclustereffect) + * 5.2.8.195 [RQ.SRS-006.RBAC.Revoke.AdminOption](#rqsrs-006rbacrevokeadminoption) + * 5.2.8.196 [RQ.SRS-006.RBAC.Revoke.AdminOption.Effect](#rqsrs-006rbacrevokeadminoptioneffect) + * 5.2.8.197 [RQ.SRS-006.RBAC.Revoke.Role.Syntax](#rqsrs-006rbacrevokerolesyntax) + * 5.2.8.198 [RQ.SRS-006.RBAC.Show.Grants](#rqsrs-006rbacshowgrants) + * 5.2.8.199 [RQ.SRS-006.RBAC.Show.Grants.For](#rqsrs-006rbacshowgrantsfor) + * 5.2.8.200 [RQ.SRS-006.RBAC.Show.Grants.Syntax](#rqsrs-006rbacshowgrantssyntax) + * 5.2.8.201 [RQ.SRS-006.RBAC.SettingsProfile.Create](#rqsrs-006rbacsettingsprofilecreate) + * 5.2.8.202 [RQ.SRS-006.RBAC.SettingsProfile.Create.Effect](#rqsrs-006rbacsettingsprofilecreateeffect) + * 5.2.8.203 [RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists](#rqsrs-006rbacsettingsprofilecreateifnotexists) + * 5.2.8.204 [RQ.SRS-006.RBAC.SettingsProfile.Create.Replace](#rqsrs-006rbacsettingsprofilecreatereplace) + * 5.2.8.205 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables](#rqsrs-006rbacsettingsprofilecreatevariables) + * 5.2.8.206 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value](#rqsrs-006rbacsettingsprofilecreatevariablesvalue) + * 5.2.8.207 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value.Effect](#rqsrs-006rbacsettingsprofilecreatevariablesvalueeffect) + * 5.2.8.208 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints](#rqsrs-006rbacsettingsprofilecreatevariablesconstraints) + * 5.2.8.209 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints.Effect](#rqsrs-006rbacsettingsprofilecreatevariablesconstraintseffect) + * 5.2.8.210 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment](#rqsrs-006rbacsettingsprofilecreateassignment) + * 5.2.8.211 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None](#rqsrs-006rbacsettingsprofilecreateassignmentnone) + * 5.2.8.212 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All](#rqsrs-006rbacsettingsprofilecreateassignmentall) + * 5.2.8.213 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilecreateassignmentallexcept) + * 5.2.8.214 [RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit](#rqsrs-006rbacsettingsprofilecreateinherit) + * 5.2.8.215 [RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster](#rqsrs-006rbacsettingsprofilecreateoncluster) + * 5.2.8.216 [RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax](#rqsrs-006rbacsettingsprofilecreatesyntax) + * 5.2.8.217 [RQ.SRS-006.RBAC.SettingsProfile.Alter](#rqsrs-006rbacsettingsprofilealter) + * 5.2.8.218 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Effect](#rqsrs-006rbacsettingsprofilealtereffect) + * 5.2.8.219 [RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists](#rqsrs-006rbacsettingsprofilealterifexists) + * 5.2.8.220 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename](#rqsrs-006rbacsettingsprofilealterrename) + * 5.2.8.221 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables](#rqsrs-006rbacsettingsprofilealtervariables) + * 5.2.8.222 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value](#rqsrs-006rbacsettingsprofilealtervariablesvalue) + * 5.2.8.223 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value.Effect](#rqsrs-006rbacsettingsprofilealtervariablesvalueeffect) + * 5.2.8.224 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints](#rqsrs-006rbacsettingsprofilealtervariablesconstraints) + * 5.2.8.225 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints.Effect](#rqsrs-006rbacsettingsprofilealtervariablesconstraintseffect) + * 5.2.8.226 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment](#rqsrs-006rbacsettingsprofilealterassignment) + * 5.2.8.227 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Effect](#rqsrs-006rbacsettingsprofilealterassignmenteffect) + * 5.2.8.228 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None](#rqsrs-006rbacsettingsprofilealterassignmentnone) + * 5.2.8.229 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All](#rqsrs-006rbacsettingsprofilealterassignmentall) + * 5.2.8.230 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilealterassignmentallexcept) + * 5.2.8.231 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit](#rqsrs-006rbacsettingsprofilealterassignmentinherit) + * 5.2.8.232 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster](#rqsrs-006rbacsettingsprofilealterassignmentoncluster) + * 5.2.8.233 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax](#rqsrs-006rbacsettingsprofilealtersyntax) + * 5.2.8.234 [RQ.SRS-006.RBAC.SettingsProfile.Drop](#rqsrs-006rbacsettingsprofiledrop) + * 5.2.8.235 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Effect](#rqsrs-006rbacsettingsprofiledropeffect) + * 5.2.8.236 [RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists](#rqsrs-006rbacsettingsprofiledropifexists) + * 5.2.8.237 [RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster](#rqsrs-006rbacsettingsprofiledroponcluster) + * 5.2.8.238 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax](#rqsrs-006rbacsettingsprofiledropsyntax) + * 5.2.8.239 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile](#rqsrs-006rbacsettingsprofileshowcreatesettingsprofile) + * 5.2.8.240 [RQ.SRS-006.RBAC.Quota.Create](#rqsrs-006rbacquotacreate) + * 5.2.8.241 [RQ.SRS-006.RBAC.Quota.Create.Effect](#rqsrs-006rbacquotacreateeffect) + * 5.2.8.242 [RQ.SRS-006.RBAC.Quota.Create.IfNotExists](#rqsrs-006rbacquotacreateifnotexists) + * 5.2.8.243 [RQ.SRS-006.RBAC.Quota.Create.Replace](#rqsrs-006rbacquotacreatereplace) + * 5.2.8.244 [RQ.SRS-006.RBAC.Quota.Create.Cluster](#rqsrs-006rbacquotacreatecluster) + * 5.2.8.245 [RQ.SRS-006.RBAC.Quota.Create.Interval](#rqsrs-006rbacquotacreateinterval) + * 5.2.8.246 [RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized](#rqsrs-006rbacquotacreateintervalrandomized) + * 5.2.8.247 [RQ.SRS-006.RBAC.Quota.Create.Queries](#rqsrs-006rbacquotacreatequeries) + * 5.2.8.248 [RQ.SRS-006.RBAC.Quota.Create.Errors](#rqsrs-006rbacquotacreateerrors) + * 5.2.8.249 [RQ.SRS-006.RBAC.Quota.Create.ResultRows](#rqsrs-006rbacquotacreateresultrows) + * 5.2.8.250 [RQ.SRS-006.RBAC.Quota.Create.ReadRows](#rqsrs-006rbacquotacreatereadrows) + * 5.2.8.251 [RQ.SRS-006.RBAC.Quota.Create.ResultBytes](#rqsrs-006rbacquotacreateresultbytes) + * 5.2.8.252 [RQ.SRS-006.RBAC.Quota.Create.ReadBytes](#rqsrs-006rbacquotacreatereadbytes) + * 5.2.8.253 [RQ.SRS-006.RBAC.Quota.Create.ExecutionTime](#rqsrs-006rbacquotacreateexecutiontime) + * 5.2.8.254 [RQ.SRS-006.RBAC.Quota.Create.NoLimits](#rqsrs-006rbacquotacreatenolimits) + * 5.2.8.255 [RQ.SRS-006.RBAC.Quota.Create.TrackingOnly](#rqsrs-006rbacquotacreatetrackingonly) + * 5.2.8.256 [RQ.SRS-006.RBAC.Quota.Create.KeyedBy](#rqsrs-006rbacquotacreatekeyedby) + * 5.2.8.257 [RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions](#rqsrs-006rbacquotacreatekeyedbyoptions) + * 5.2.8.258 [RQ.SRS-006.RBAC.Quota.Create.Assignment](#rqsrs-006rbacquotacreateassignment) + * 5.2.8.259 [RQ.SRS-006.RBAC.Quota.Create.Assignment.None](#rqsrs-006rbacquotacreateassignmentnone) + * 5.2.8.260 [RQ.SRS-006.RBAC.Quota.Create.Assignment.All](#rqsrs-006rbacquotacreateassignmentall) + * 5.2.8.261 [RQ.SRS-006.RBAC.Quota.Create.Assignment.Except](#rqsrs-006rbacquotacreateassignmentexcept) + * 5.2.8.262 [RQ.SRS-006.RBAC.Quota.Create.Syntax](#rqsrs-006rbacquotacreatesyntax) + * 5.2.8.263 [RQ.SRS-006.RBAC.Quota.Alter](#rqsrs-006rbacquotaalter) + * 5.2.8.264 [RQ.SRS-006.RBAC.Quota.Alter.Effect](#rqsrs-006rbacquotaaltereffect) + * 5.2.8.265 [RQ.SRS-006.RBAC.Quota.Alter.IfExists](#rqsrs-006rbacquotaalterifexists) + * 5.2.8.266 [RQ.SRS-006.RBAC.Quota.Alter.Rename](#rqsrs-006rbacquotaalterrename) + * 5.2.8.267 [RQ.SRS-006.RBAC.Quota.Alter.Cluster](#rqsrs-006rbacquotaaltercluster) + * 5.2.8.268 [RQ.SRS-006.RBAC.Quota.Alter.Interval](#rqsrs-006rbacquotaalterinterval) + * 5.2.8.269 [RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized](#rqsrs-006rbacquotaalterintervalrandomized) + * 5.2.8.270 [RQ.SRS-006.RBAC.Quota.Alter.Queries](#rqsrs-006rbacquotaalterqueries) + * 5.2.8.271 [RQ.SRS-006.RBAC.Quota.Alter.Errors](#rqsrs-006rbacquotaaltererrors) + * 5.2.8.272 [RQ.SRS-006.RBAC.Quota.Alter.ResultRows](#rqsrs-006rbacquotaalterresultrows) + * 5.2.8.273 [RQ.SRS-006.RBAC.Quota.Alter.ReadRows](#rqsrs-006rbacquotaalterreadrows) + * 5.2.8.274 [RQ.SRS-006.RBAC.Quota.ALter.ResultBytes](#rqsrs-006rbacquotaalterresultbytes) + * 5.2.8.275 [RQ.SRS-006.RBAC.Quota.Alter.ReadBytes](#rqsrs-006rbacquotaalterreadbytes) + * 5.2.8.276 [RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime](#rqsrs-006rbacquotaalterexecutiontime) + * 5.2.8.277 [RQ.SRS-006.RBAC.Quota.Alter.NoLimits](#rqsrs-006rbacquotaalternolimits) + * 5.2.8.278 [RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly](#rqsrs-006rbacquotaaltertrackingonly) + * 5.2.8.279 [RQ.SRS-006.RBAC.Quota.Alter.KeyedBy](#rqsrs-006rbacquotaalterkeyedby) + * 5.2.8.280 [RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions](#rqsrs-006rbacquotaalterkeyedbyoptions) + * 5.2.8.281 [RQ.SRS-006.RBAC.Quota.Alter.Assignment](#rqsrs-006rbacquotaalterassignment) + * 5.2.8.282 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.None](#rqsrs-006rbacquotaalterassignmentnone) + * 5.2.8.283 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.All](#rqsrs-006rbacquotaalterassignmentall) + * 5.2.8.284 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except](#rqsrs-006rbacquotaalterassignmentexcept) + * 5.2.8.285 [RQ.SRS-006.RBAC.Quota.Alter.Syntax](#rqsrs-006rbacquotaaltersyntax) + * 5.2.8.286 [RQ.SRS-006.RBAC.Quota.Drop](#rqsrs-006rbacquotadrop) + * 5.2.8.287 [RQ.SRS-006.RBAC.Quota.Drop.Effect](#rqsrs-006rbacquotadropeffect) + * 5.2.8.288 [RQ.SRS-006.RBAC.Quota.Drop.IfExists](#rqsrs-006rbacquotadropifexists) + * 5.2.8.289 [RQ.SRS-006.RBAC.Quota.Drop.Cluster](#rqsrs-006rbacquotadropcluster) + * 5.2.8.290 [RQ.SRS-006.RBAC.Quota.Drop.Syntax](#rqsrs-006rbacquotadropsyntax) + * 5.2.8.291 [RQ.SRS-006.RBAC.Quota.ShowQuotas](#rqsrs-006rbacquotashowquotas) + * 5.2.8.292 [RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile](#rqsrs-006rbacquotashowquotasintooutfile) + * 5.2.8.293 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Format](#rqsrs-006rbacquotashowquotasformat) + * 5.2.8.294 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings](#rqsrs-006rbacquotashowquotassettings) + * 5.2.8.295 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax](#rqsrs-006rbacquotashowquotassyntax) + * 5.2.8.296 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name](#rqsrs-006rbacquotashowcreatequotaname) + * 5.2.8.297 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current](#rqsrs-006rbacquotashowcreatequotacurrent) + * 5.2.8.298 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax](#rqsrs-006rbacquotashowcreatequotasyntax) + * 5.2.8.299 [RQ.SRS-006.RBAC.RowPolicy.Create](#rqsrs-006rbacrowpolicycreate) + * 5.2.8.300 [RQ.SRS-006.RBAC.RowPolicy.Create.Effect](#rqsrs-006rbacrowpolicycreateeffect) + * 5.2.8.301 [RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists](#rqsrs-006rbacrowpolicycreateifnotexists) + * 5.2.8.302 [RQ.SRS-006.RBAC.RowPolicy.Create.Replace](#rqsrs-006rbacrowpolicycreatereplace) + * 5.2.8.303 [RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster](#rqsrs-006rbacrowpolicycreateoncluster) + * 5.2.8.304 [RQ.SRS-006.RBAC.RowPolicy.Create.On](#rqsrs-006rbacrowpolicycreateon) + * 5.2.8.305 [RQ.SRS-006.RBAC.RowPolicy.Create.Access](#rqsrs-006rbacrowpolicycreateaccess) + * 5.2.8.306 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive](#rqsrs-006rbacrowpolicycreateaccesspermissive) + * 5.2.8.307 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive](#rqsrs-006rbacrowpolicycreateaccessrestrictive) + * 5.2.8.308 [RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect](#rqsrs-006rbacrowpolicycreateforselect) + * 5.2.8.309 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition](#rqsrs-006rbacrowpolicycreatecondition) + * 5.2.8.310 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition.Effect](#rqsrs-006rbacrowpolicycreateconditioneffect) + * 5.2.8.311 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment](#rqsrs-006rbacrowpolicycreateassignment) + * 5.2.8.312 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None](#rqsrs-006rbacrowpolicycreateassignmentnone) + * 5.2.8.313 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All](#rqsrs-006rbacrowpolicycreateassignmentall) + * 5.2.8.314 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept](#rqsrs-006rbacrowpolicycreateassignmentallexcept) + * 5.2.8.315 [RQ.SRS-006.RBAC.RowPolicy.Create.Syntax](#rqsrs-006rbacrowpolicycreatesyntax) + * 5.2.8.316 [RQ.SRS-006.RBAC.RowPolicy.Alter](#rqsrs-006rbacrowpolicyalter) + * 5.2.8.317 [RQ.SRS-006.RBAC.RowPolicy.Alter.Effect](#rqsrs-006rbacrowpolicyaltereffect) + * 5.2.8.318 [RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists](#rqsrs-006rbacrowpolicyalterifexists) + * 5.2.8.319 [RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect](#rqsrs-006rbacrowpolicyalterforselect) + * 5.2.8.320 [RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster](#rqsrs-006rbacrowpolicyalteroncluster) + * 5.2.8.321 [RQ.SRS-006.RBAC.RowPolicy.Alter.On](#rqsrs-006rbacrowpolicyalteron) + * 5.2.8.322 [RQ.SRS-006.RBAC.RowPolicy.Alter.Rename](#rqsrs-006rbacrowpolicyalterrename) + * 5.2.8.323 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access](#rqsrs-006rbacrowpolicyalteraccess) + * 5.2.8.324 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive](#rqsrs-006rbacrowpolicyalteraccesspermissive) + * 5.2.8.325 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive](#rqsrs-006rbacrowpolicyalteraccessrestrictive) + * 5.2.8.326 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition](#rqsrs-006rbacrowpolicyaltercondition) + * 5.2.8.327 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.Effect](#rqsrs-006rbacrowpolicyalterconditioneffect) + * 5.2.8.328 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None](#rqsrs-006rbacrowpolicyalterconditionnone) + * 5.2.8.329 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment](#rqsrs-006rbacrowpolicyalterassignment) + * 5.2.8.330 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None](#rqsrs-006rbacrowpolicyalterassignmentnone) + * 5.2.8.331 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All](#rqsrs-006rbacrowpolicyalterassignmentall) + * 5.2.8.332 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept](#rqsrs-006rbacrowpolicyalterassignmentallexcept) + * 5.2.8.333 [RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax](#rqsrs-006rbacrowpolicyaltersyntax) + * 5.2.8.334 [RQ.SRS-006.RBAC.RowPolicy.Drop](#rqsrs-006rbacrowpolicydrop) + * 5.2.8.335 [RQ.SRS-006.RBAC.RowPolicy.Drop.Effect](#rqsrs-006rbacrowpolicydropeffect) + * 5.2.8.336 [RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists](#rqsrs-006rbacrowpolicydropifexists) + * 5.2.8.337 [RQ.SRS-006.RBAC.RowPolicy.Drop.On](#rqsrs-006rbacrowpolicydropon) + * 5.2.8.338 [RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster](#rqsrs-006rbacrowpolicydroponcluster) + * 5.2.8.339 [RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax](#rqsrs-006rbacrowpolicydropsyntax) + * 5.2.8.340 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy](#rqsrs-006rbacrowpolicyshowcreaterowpolicy) + * 5.2.8.341 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On](#rqsrs-006rbacrowpolicyshowcreaterowpolicyon) + * 5.2.8.342 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax](#rqsrs-006rbacrowpolicyshowcreaterowpolicysyntax) + * 5.2.8.343 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies](#rqsrs-006rbacrowpolicyshowrowpolicies) + * 5.2.8.344 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On](#rqsrs-006rbacrowpolicyshowrowpolicieson) + * 5.2.8.345 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) + * 5.2.9 [Privileges](#privileges) + * 5.2.9.1 [RQ.SRS-006.RBAC.Privileges.Usage](#rqsrs-006rbacprivilegesusage) + * 5.2.9.2 [Select](#select) + * 5.2.9.2.1 [RQ.SRS-006.RBAC.Privileges.Select](#rqsrs-006rbacprivilegesselect) + * 5.2.9.2.2 [RQ.SRS-006.RBAC.Privileges.Select.Grant](#rqsrs-006rbacprivilegesselectgrant) + * 5.2.9.2.3 [RQ.SRS-006.RBAC.Privileges.Select.Revoke](#rqsrs-006rbacprivilegesselectrevoke) + * 5.2.9.2.4 [RQ.SRS-006.RBAC.Privileges.Select.Column](#rqsrs-006rbacprivilegesselectcolumn) + * 5.2.9.2.5 [RQ.SRS-006.RBAC.Privileges.Select.Cluster](#rqsrs-006rbacprivilegesselectcluster) + * 5.2.9.2.6 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption](#rqsrs-006rbacprivilegesselectgrantoption) + * 5.2.9.2.7 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant](#rqsrs-006rbacprivilegesselectgrantoptiongrant) + * 5.2.9.2.8 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke](#rqsrs-006rbacprivilegesselectgrantoptionrevoke) + * 5.2.9.3 [Insert](#insert) + * 5.2.9.3.1 [RQ.SRS-006.RBAC.Privileges.Insert](#rqsrs-006rbacprivilegesinsert) + * 5.2.9.3.2 [RQ.SRS-006.RBAC.Privileges.Insert.Grant](#rqsrs-006rbacprivilegesinsertgrant) + * 5.2.9.3.3 [RQ.SRS-006.RBAC.Privileges.Insert.Revoke](#rqsrs-006rbacprivilegesinsertrevoke) + * 5.2.9.3.4 [RQ.SRS-006.RBAC.Privileges.Insert.Column](#rqsrs-006rbacprivilegesinsertcolumn) + * 5.2.9.3.5 [RQ.SRS-006.RBAC.Privileges.Insert.Cluster](#rqsrs-006rbacprivilegesinsertcluster) + * 5.2.9.3.6 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption](#rqsrs-006rbacprivilegesinsertgrantoption) + * 5.2.9.3.7 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant](#rqsrs-006rbacprivilegesinsertgrantoptiongrant) + * 5.2.9.3.8 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke](#rqsrs-006rbacprivilegesinsertgrantoptionrevoke) + * 5.2.9.4 [RQ.SRS-006.RBAC.Privileges.Delete](#rqsrs-006rbacprivilegesdelete) + * 5.2.9.5 [RQ.SRS-006.RBAC.Privileges.Alter](#rqsrs-006rbacprivilegesalter) + * 5.2.9.6 [RQ.SRS-006.RBAC.Privileges.Create](#rqsrs-006rbacprivilegescreate) + * 5.2.9.7 [RQ.SRS-006.RBAC.Privileges.Drop](#rqsrs-006rbacprivilegesdrop) + * 5.2.9.8 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.9.9 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) + * 5.2.9.10 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.9.11 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) + * 5.2.10 [Required Privileges](#required-privileges) + * 5.2.10.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) + * 5.2.10.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) + * 5.2.10.3 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop](#rqsrs-006rbacrequiredprivilegesdrop) + * 5.2.10.4 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table](#rqsrs-006rbacrequiredprivilegesdroptable) + * 5.2.10.5 [RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke](#rqsrs-006rbacrequiredprivilegesgrantrevoke) + * 5.2.10.6 [RQ.SRS-006.RBAC.RequiredPrivileges.Use](#rqsrs-006rbacrequiredprivilegesuse) + * 5.2.10.7 [RQ.SRS-006.RBAC.RequiredPrivileges.Admin](#rqsrs-006rbacrequiredprivilegesadmin) * 6 [References](#references) ## Revision History This document is stored in an electronic form using [Git] source control management software -hosted in a Gitlab repository. +hosted in a GitHub repository. All the updates are tracked using the [Git]'s revision history. @@ -475,8 +482,6 @@ to [ClickHouse]. privilege to access a database or a table * **select** - privilege to read data from a database or a table -* **select columns** - - privilege to read specific columns from a table * **insert** privilege to insert data into a database or a table * **delete** @@ -488,7 +493,7 @@ to [ClickHouse]. * **drop** privilege to drop a database or a table * **all** - privilege that includes **usage**, **select**, **select columns**, + privilege that includes **usage**, **select**, **insert**, **delete**, **alter**, **create**, and **drop** * **grant option** privilege to grant the same privilege to other users or roles @@ -632,165 +637,6 @@ version: 1.0 [ClickHouse] SHALL support assigning one or more **row policies** to a **role**. -#### Privileges - -##### RQ.SRS-006.RBAC.Privileges.Usage -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **usage** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.Select -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **select** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.SelectColumns -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **select columns** privilege -for a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.Insert -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **insert** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.Delete -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **delete** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.Alter -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **alter** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.Create -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **create** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.Drop -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **drop** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.All -version: 1.0 - -[ClickHouse] SHALL include in the **all** privilege the same rights -as provided by **usage**, **select**, **select columns**, -**insert**, **delete**, **alter**, **create**, and **drop** privileges. - -##### RQ.SRS-006.RBAC.Privileges.All.GrantRevoke -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **all** privileges -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **grant option** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.AdminOption -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **admin option** privilege -to one or more **users** or **roles**. - -#### Required Privileges - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Insert -version: 1.0 - -[ClickHouse] SHALL not allow any `INSERT INTO` statements -to be executed unless the user has the **insert** privilege for the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Select -version: 1.0 - -[ClickHouse] SHALL not allow any `SELECT` statements -to be executed unless the user has the **select** or **select columns** privilege -for the destination table either because of the explicit grant -or through one of the roles assigned to the user. -If the the user only has the **select columns** -privilege then only the specified columns SHALL be available for reading. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Create -version: 1.0 - -[ClickHouse] SHALL not allow any `CREATE` statements -to be executed unless the user has the **create** privilege for the destination database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Alter -version: 1.0 - -[ClickHouse] SHALL not allow any `ALTER` statements -to be executed unless the user has the **alter** privilege for the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop -version: 1.0 - -[ClickHouse] SHALL not allow any `DROP` statements -to be executed unless the user has the **drop** privilege for the destination database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table -version: 1.0 - -[ClickHouse] SHALL not allow any `DROP TABLE` statements -to be executed unless the user has the **drop** privilege for the destination database or the table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke -version: 1.0 - -[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements -to be executed unless the user has the **grant option** privilege -for the privilege of the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Use -version: 1.0 - -[ClickHouse] SHALL not allow the `USE` statement to be executed -unless the user has at least one of the privileges for the database -or the table inside that database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Admin -version: 1.0 - -[ClickHouse] SHALL not allow any of the following statements - -* `SYSTEM` -* `SHOW` -* `ATTACH` -* `CHECK TABLE` -* `DESCRIBE TABLE` -* `DETACH` -* `EXISTS` -* `KILL QUERY` -* `KILL MUTATION` -* `OPTIMIZE` -* `RENAME` -* `TRUNCATE` - -to be executed unless the user has the **admin option** privilege -through one of the roles with **admin option** privilege assigned to the user. - #### Partial Revokes ##### RQ.SRS-006.RBAC.PartialRevokes @@ -1556,47 +1402,20 @@ version: 1.0 [ClickHouse] SHALL support granting privileges to current user using `TO CURRENT_USER` clause in the `GRANT PRIVILEGE` statement. - ##### RQ.SRS-006.RBAC.Grant.Privilege.Select version: 1.0 [ClickHouse] SHALL support granting the **select** privilege to one or more users or roles for a database or a table using the `GRANT SELECT` statement. -##### RQ.SRS-006.RBAC.Grant.Privilege.Select.Effect -version: 1.0 - -[ClickHouse] SHALL add the **select** privilege to the specified users or roles -after the successful execution of the `GRANT SELECT` statement. -Any new operation by a user or a user that has the specified role -which requires the **select** privilege SHALL succeed. - -##### RQ.SRS-006.RBAC.Grant.Privilege.SelectColumns -version: 1.0 - -[ClickHouse] SHALL support granting the **select columns** privilege to one or more users or roles -for a database or a table using the `GRANT SELECT(columns)` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.SelectColumns.Effect -version: 1.0 - -[ClickHouse] SHALL add the **select columns** privilege to the specified users or roles -after the successful execution of the `GRANT SELECT(columns)` statement. -Any new operation by a user or a user that has the specified role -which requires the **select columns** privilege SHALL succeed. - ##### RQ.SRS-006.RBAC.Grant.Privilege.Insert version: 1.0 [ClickHouse] SHALL support granting the **insert** privilege to one or more users or roles for a database or a table using the `GRANT INSERT` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Insert.Effect -version: 1.0 - -[ClickHouse] SHALL add the **insert** privilege to the specified users or roles -after the successful execution of the `GRANT INSERT` statement. -Any new operation by a user or a user that has the specified role +When the **insert** privilege is granted to the specified users or roles +after the successful execution of the `GRANT INSERT` statement, +any new operation by a user or a user that has the specified role which requires the **insert** privilege SHALL succeed. ##### RQ.SRS-006.RBAC.Grant.Privilege.Alter @@ -1861,9 +1680,8 @@ version: 1.0 grants explicit privileges to a user or a role. ```sql -GRANT [ON CLUSTER cluster_name] - privilege {SELECT | SELECT(columns) | INSERT | ALTER | CREATE | DROP | TRUNCATE | OPTIMIZE | SHOW | KILL QUERY | ACCESS MANAGEMENT | SYSTEM | INTROSPECTION | SOURCES | dictGet | NONE |ALL [PRIVILEGES]} [, ...] - ON {*.* | database.* | database.table | * | table} +GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] + ON {db.table|db.*|*.*|table|*} TO {user | role | CURRENT_USER} [,...] [WITH GRANT OPTION] ``` @@ -1905,26 +1723,14 @@ version: 1.0 [ClickHouse] SHALL support revoking the **select** privilege to one or more users or roles for a database or a table using the `REVOKE SELECT` statement. -##### RQ.SRS-006.RBAC.Revoke.Privilege.Select.Effect -version: 1.0 - -[ClickHouse] SHALL remove the **select** privilege from the specified users or roles -after the successful execution of the `REVOKE SELECT` statement. -Any new operation by a user or a user that had the specified role -which requires the **select** privilege SHALL fail if user does not have it otherwise. - ##### RQ.SRS-006.RBAC.Revoke.Privilege.Insert version: 1.0 [ClickHouse] SHALL support revoking the **insert** privilege to one or more users or roles for a database or a table using the `REVOKE INSERT` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Insert.Effect -version: 1.0 - -[ClickHouse] SHALL remove the **insert** privilege from the specified users or roles -after the successful execution of the `REVOKE INSERT` statement. -Any new operation by a user or a user that had the specified role +When the **insert** privilege is removed from the specified users or roles +after the successful execution of the `REVOKE INSERT` statement, +any new operation by a user or a user that had the specified role which requires the **insert** privilege SHALL fail if user does not have it otherwise. ##### RQ.SRS-006.RBAC.Revoke.Privilege.Alter @@ -2230,13 +2036,6 @@ REVOKE [ON CLUSTER cluster_name] privilege ON {db.table|db.*|*.*|table|*} FROM {user | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user | CURRENT_USER} [,...] ``` - - ##### RQ.SRS-006.RBAC.PartialRevoke.Syntax version: 1.0 @@ -3421,6 +3220,264 @@ version: 1.0 SHOW [ROW] POLICIES [ON [database.]table] ``` +#### Privileges + +##### RQ.SRS-006.RBAC.Privileges.Usage +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **usage** privilege +for a database or a specific table to one or more **users** or **roles**. + +##### Select + +###### RQ.SRS-006.RBAC.Privileges.Select +version: 1.0 + +[ClickHouse] SHALL support changing access to the **select** privilege +for a database or a specific table to one or more **users** or **roles**. +Any `SELECT INTO` statements SHALL not to be executed, unless the user +has the **select** privilege for the destination table +either because of the explicit grant or through one of the roles assigned to the user. + +###### RQ.SRS-006.RBAC.Privileges.Select.Grant +version: 1.0 + +[ClickHouse] SHALL support granting **select** privilege +for a database or a specific table to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.Select.Revoke +version: 1.0 + +[ClickHouse] SHALL support revoking **select** privilege +for a database or a specific table to one or more **users** or **roles** + +###### RQ.SRS-006.RBAC.Privileges.Select.Column +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **select** privilege +for one or more specified columns in a table to one or more **users** or **roles**. +Any `SELECT INTO` statements SHALL not to be executed, unless the user +has the **select** privilege for the destination column +either because of the explicit grant or through one of the roles assigned to the user. + +###### RQ.SRS-006.RBAC.Privileges.Select.Cluster +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **select** privilege +on a specified cluster to one or more **users** or **roles**. +Any `SELECT INTO` statements SHALL succeed only on nodes where +the table exists and privilege was granted. + +###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption +version: 1.0 + +[ClickHouse] SHALL support granting **select** privilege +for a database or a specific table to one or more **users** or **roles** +with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to +change access to the **select** privilege by another user or role +on the same or smaller scope that they have access to. + +###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant +version: 1.0 + +[ClickHouse] SHALL support a user with **grant option** privilege +granting **select** privilege to other **users** or **roles** on the same +or smaller scope that they have access to. Any `SELECT INTO` statements SHALL succeed +when done by a user with privilege granted by a user with `GRANT OPTION`, +either directly or through an assigned role. + +###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke +version: 1.0 + +[ClickHouse] SHALL support a user with **grant option** privilege +revoking **select** privilege from other **users** or **roles** on the same +or smaller scope that they have access to. Any `SELECT INTO` statements SHALL fail +when done by a user with privilege revoke by a user with `GRANT OPTION`, +either directly or through an assigned role, unless they have access otherwise. + +##### Insert + +###### RQ.SRS-006.RBAC.Privileges.Insert +version: 1.0 + +[ClickHouse] SHALL support changing access to the **insert** privilege +for a database or a specific table to one or more **users** or **roles**. +Any `INSERT INTO` statements SHALL not to be executed, unless the user +has the **insert** privilege for the destination table +either because of the explicit grant or through one of the roles assigned to the user. + +###### RQ.SRS-006.RBAC.Privileges.Insert.Grant +version: 1.0 + +[ClickHouse] SHALL support granting **insert** privilege +for a database or a specific table to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.Insert.Revoke +version: 1.0 + +[ClickHouse] SHALL support revoking **insert** privilege +for a database or a specific table to one or more **users** or **roles** + +###### RQ.SRS-006.RBAC.Privileges.Insert.Column +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **insert** privilege +for one or more specified columns in a table to one or more **users** or **roles**. +Any `INSERT INTO` statements SHALL not to be executed, unless the user +has the **insert** privilege for the destination column +either because of the explicit grant or through one of the roles assigned to the user. + +###### RQ.SRS-006.RBAC.Privileges.Insert.Cluster +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **insert** privilege +on a specified cluster to one or more **users** or **roles**. +Any `INSERT INTO` statements SHALL succeed only on nodes where +the table exists and privilege was granted. + +###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption +version: 1.0 + +[ClickHouse] SHALL support granting **insert** privilege +for a database or a specific table to one or more **users** or **roles** +with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to +change access to the **insert** privilege by another user or role +on the same or smaller scope that they have access to. + +###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant +version: 1.0 + +[ClickHouse] SHALL support a user with **grant option** privilege +granting **insert** privilege to other **users** or **roles** on the same +or smaller scope that they have access to. Any `INSERT INTO` statements SHALL succeed +when done by a user with privilege granted by a user with `GRANT OPTION`, +either directly or through an assigned role. + +###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke +version: 1.0 + +[ClickHouse] SHALL support a user with **grant option** privilege +revoking **insert** privilege from other **users** or **roles** on the same +or smaller scope that they have access to. Any `INSERT INTO` statements SHALL fail +when done by a user with privilege revoke by a user with `GRANT OPTION`, +either directly or through an assigned role, unless they have access otherwise. + +##### RQ.SRS-006.RBAC.Privileges.Delete +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **delete** privilege +for a database or a specific table to one or more **users** or **roles**. + +##### RQ.SRS-006.RBAC.Privileges.Alter +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter** privilege +for a database or a specific table to one or more **users** or **roles**. + +##### RQ.SRS-006.RBAC.Privileges.Create +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **create** privilege +for a database or a specific table to one or more **users** or **roles**. + +##### RQ.SRS-006.RBAC.Privileges.Drop +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **drop** privilege +for a database or a specific table to one or more **users** or **roles**. + +##### RQ.SRS-006.RBAC.Privileges.All +version: 1.0 + +[ClickHouse] SHALL include in the **all** privilege the same rights +as provided by **usage**, **select**, **select columns**, +**insert**, **delete**, **alter**, **create**, and **drop** privileges. + +##### RQ.SRS-006.RBAC.Privileges.All.GrantRevoke +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **all** privileges +for a database or a specific table to one or more **users** or **roles**. + +##### RQ.SRS-006.RBAC.Privileges.GrantOption +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **grant option** privilege +for a database or a specific table to one or more **users** or **roles**. + +##### RQ.SRS-006.RBAC.Privileges.AdminOption +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **admin option** privilege +to one or more **users** or **roles**. + +#### Required Privileges + +##### RQ.SRS-006.RBAC.RequiredPrivileges.Create +version: 1.0 + +[ClickHouse] SHALL not allow any `CREATE` statements +to be executed unless the user has the **create** privilege for the destination database +either because of the explicit grant or through one of the roles assigned to the user. + +##### RQ.SRS-006.RBAC.RequiredPrivileges.Alter +version: 1.0 + +[ClickHouse] SHALL not allow any `ALTER` statements +to be executed unless the user has the **alter** privilege for the destination table +either because of the explicit grant or through one of the roles assigned to the user. + +##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop +version: 1.0 + +[ClickHouse] SHALL not allow any `DROP` statements +to be executed unless the user has the **drop** privilege for the destination database +either because of the explicit grant or through one of the roles assigned to the user. + +##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table +version: 1.0 + +[ClickHouse] SHALL not allow any `DROP TABLE` statements +to be executed unless the user has the **drop** privilege for the destination database or the table +either because of the explicit grant or through one of the roles assigned to the user. + +##### RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke +version: 1.0 + +[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements +to be executed unless the user has the **grant option** privilege +for the privilege of the destination table +either because of the explicit grant or through one of the roles assigned to the user. + +##### RQ.SRS-006.RBAC.RequiredPrivileges.Use +version: 1.0 + +[ClickHouse] SHALL not allow the `USE` statement to be executed +unless the user has at least one of the privileges for the database +or the table inside that database +either because of the explicit grant or through one of the roles assigned to the user. + +##### RQ.SRS-006.RBAC.RequiredPrivileges.Admin +version: 1.0 + +[ClickHouse] SHALL not allow any of the following statements + +* `SYSTEM` +* `SHOW` +* `ATTACH` +* `CHECK TABLE` +* `DESCRIBE TABLE` +* `DETACH` +* `EXISTS` +* `KILL QUERY` +* `KILL MUTATION` +* `OPTIMIZE` +* `RENAME` +* `TRUNCATE` + +to be executed unless the user has the **admin option** privilege +through one of the roles with **admin option** privilege assigned to the user. + ## References * **ClickHouse:** https://clickhouse.tech diff --git a/tests/testflows/rbac/requirements/requirements.py b/tests/testflows/rbac/requirements/requirements.py index 02cb53bf64c..7b2eedfe60b 100644 --- a/tests/testflows/rbac/requirements/requirements.py +++ b/tests/testflows/rbac/requirements/requirements.py @@ -1,6 +1,6 @@ # These requirements were auto generated # from software requirements specification (SRS) -# document by TestFlows v1.6.200723.1011705. +# document by TestFlows v1.6.200731.1222107. # Do not edit by hand but re-generate instead # using 'tfs requirements generate' command. from testflows.core import Requirement @@ -316,329 +316,6 @@ RQ_SRS_006_RBAC_Role_RowPolicies = Requirement( link=None ) -RQ_SRS_006_RBAC_Privileges_Usage = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Usage', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **usage** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_Select = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **select** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_SelectColumns = Requirement( - name='RQ.SRS-006.RBAC.Privileges.SelectColumns', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **select columns** privilege\n' - 'for a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_Insert = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_Delete = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Delete', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **delete** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_Alter = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_Create = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **create** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_Drop = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **drop** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_All = Requirement( - name='RQ.SRS-006.RBAC.Privileges.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL include in the **all** privilege the same rights\n' - 'as provided by **usage**, **select**, **select columns**,\n' - '**insert**, **delete**, **alter**, **create**, and **drop** privileges.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_All_GrantRevoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.All.GrantRevoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **all** privileges\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **grant option** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Privileges_AdminOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AdminOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **admin option** privilege\n' - 'to one or more **users** or **roles**.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Insert = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `INSERT INTO` statements\n' - 'to be executed unless the user has the **insert** privilege for the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Select = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `SELECT` statements\n' - 'to be executed unless the user has the **select** or **select columns** privilege\n' - 'for the destination table either because of the explicit grant\n' - 'or through one of the roles assigned to the user.\n' - 'If the the user only has the **select columns**\n' - 'privilege then only the specified columns SHALL be available for reading.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Create = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `CREATE` statements\n' - 'to be executed unless the user has the **create** privilege for the destination database\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Alter = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `ALTER` statements\n' - 'to be executed unless the user has the **alter** privilege for the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Drop = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `DROP` statements\n' - 'to be executed unless the user has the **drop** privilege for the destination database\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Drop_Table = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `DROP TABLE` statements\n' - 'to be executed unless the user has the **drop** privilege for the destination database or the table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_GrantRevoke = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements\n' - 'to be executed unless the user has the **grant option** privilege\n' - 'for the privilege of the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Use = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Use', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow the `USE` statement to be executed\n' - 'unless the user has at least one of the privileges for the database\n' - 'or the table inside that database\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_RequiredPrivileges_Admin = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Admin', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any of the following statements\n' - '\n' - '* `SYSTEM`\n' - '* `SHOW`\n' - '* `ATTACH`\n' - '* `CHECK TABLE`\n' - '* `DESCRIBE TABLE`\n' - '* `DETACH`\n' - '* `EXISTS`\n' - '* `KILL QUERY`\n' - '* `KILL MUTATION`\n' - '* `OPTIMIZE`\n' - '* `RENAME`\n' - '* `TRUNCATE`\n' - '\n' - 'to be executed unless the user has the **admin option** privilege\n' - 'through one of the roles with **admin option** privilege assigned to the user.\n' - ), - link=None - ) - RQ_SRS_006_RBAC_PartialRevokes = Requirement( name='RQ.SRS-006.RBAC.PartialRevokes', version='1.0', @@ -2319,52 +1996,6 @@ RQ_SRS_006_RBAC_Grant_Privilege_Select = Requirement( link=None ) -RQ_SRS_006_RBAC_Grant_Privilege_Select_Effect = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Select.Effect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL add the **select** privilege to the specified users or roles\n' - 'after the successful execution of the `GRANT SELECT` statement.\n' - 'Any new operation by a user or a user that has the specified role\n' - 'which requires the **select** privilege SHALL succeed.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Grant_Privilege_SelectColumns = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.SelectColumns', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **select columns** privilege to one or more users or roles\n' - 'for a database or a table using the `GRANT SELECT(columns)` statement.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Grant_Privilege_SelectColumns_Effect = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.SelectColumns.Effect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL add the **select columns** privilege to the specified users or roles\n' - 'after the successful execution of the `GRANT SELECT(columns)` statement.\n' - 'Any new operation by a user or a user that has the specified role\n' - 'which requires the **select columns** privilege SHALL succeed.\n' - ), - link=None - ) - RQ_SRS_006_RBAC_Grant_Privilege_Insert = Requirement( name='RQ.SRS-006.RBAC.Grant.Privilege.Insert', version='1.0', @@ -2375,21 +2006,9 @@ RQ_SRS_006_RBAC_Grant_Privilege_Insert = Requirement( description=( '[ClickHouse] SHALL support granting the **insert** privilege to one or more users or roles\n' 'for a database or a table using the `GRANT INSERT` statement.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Grant_Privilege_Insert_Effect = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Insert.Effect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL add the **insert** privilege to the specified users or roles\n' - 'after the successful execution of the `GRANT INSERT` statement.\n' - 'Any new operation by a user or a user that has the specified role\n' + 'When the **insert** privilege is granted to the specified users or roles\n' + 'after the successful execution of the `GRANT INSERT` statement,\n' + 'any new operation by a user or a user that has the specified role\n' 'which requires the **insert** privilege SHALL succeed.\n' ), link=None @@ -2942,9 +2561,8 @@ RQ_SRS_006_RBAC_Grant_Privilege_Syntax = Requirement( 'grants explicit privileges to a user or a role.\n' '\n' '```sql\n' - 'GRANT [ON CLUSTER cluster_name]\n' - ' privilege {SELECT | SELECT(columns) | INSERT | ALTER | CREATE | DROP | TRUNCATE | OPTIMIZE | SHOW | KILL QUERY | ACCESS MANAGEMENT | SYSTEM | INTROSPECTION | SOURCES | dictGet | NONE |ALL \t[PRIVILEGES]} [, ...]\n' - ' ON {*.* | database.* | database.table | * | table}\n' + 'GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...]\n' + ' ON {db.table|db.*|*.*|table|*}\n' ' TO {user | role | CURRENT_USER} [,...]\n' ' [WITH GRANT OPTION]\n' '```\n' @@ -3029,22 +2647,6 @@ RQ_SRS_006_RBAC_Revoke_Privilege_Select = Requirement( link=None ) -RQ_SRS_006_RBAC_Revoke_Privilege_Select_Effect = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Select.Effect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL remove the **select** privilege from the specified users or roles\n' - 'after the successful execution of the `REVOKE SELECT` statement.\n' - 'Any new operation by a user or a user that had the specified role\n' - 'which requires the **select** privilege SHALL fail if user does not have it otherwise.\n' - ), - link=None - ) - RQ_SRS_006_RBAC_Revoke_Privilege_Insert = Requirement( name='RQ.SRS-006.RBAC.Revoke.Privilege.Insert', version='1.0', @@ -3055,21 +2657,9 @@ RQ_SRS_006_RBAC_Revoke_Privilege_Insert = Requirement( description=( '[ClickHouse] SHALL support revoking the **insert** privilege to one or more users or roles\n' 'for a database or a table using the `REVOKE INSERT` statement.\n' - ), - link=None - ) - -RQ_SRS_006_RBAC_Revoke_Privilege_Insert_Effect = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Insert.Effect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL remove the **insert** privilege from the specified users or roles\n' - 'after the successful execution of the `REVOKE INSERT` statement.\n' - 'Any new operation by a user or a user that had the specified role\n' + 'When the **insert** privilege is removed from the specified users or roles\n' + 'after the successful execution of the `REVOKE INSERT` statement,\n' + 'any new operation by a user or a user that had the specified role\n' 'which requires the **insert** privilege SHALL fail if user does not have it otherwise.\n' ), link=None @@ -3664,13 +3254,6 @@ RQ_SRS_006_RBAC_Revoke_Privilege_Syntax = Requirement( ' ON {db.table|db.*|*.*|table|*}\n' ' FROM {user | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user | CURRENT_USER} [,...]\n' '```\n' - '\n' - '\n' ), link=None ) @@ -6192,3 +5775,509 @@ RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_Syntax = Requirement( ), link=None ) + +RQ_SRS_006_RBAC_Privileges_Usage = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Usage', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **usage** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support changing access to the **select** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `SELECT INTO` statements SHALL not to be executed, unless the user\n' + 'has the **select** privilege for the destination table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **select** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **select** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_Column = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.Column', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **select** privilege\n' + 'for one or more specified columns in a table to one or more **users** or **roles**.\n' + 'Any `SELECT INTO` statements SHALL not to be executed, unless the user\n' + 'has the **select** privilege for the destination column\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **select** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `SELECT INTO` statements SHALL succeed only on nodes where\n' + 'the table exists and privilege was granted.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_GrantOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.GrantOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **select** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' + 'change access to the **select** privilege by another user or role\n' + 'on the same or smaller scope that they have access to.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support a user with **grant option** privilege\n' + 'granting **select** privilege to other **users** or **roles** on the same\n' + 'or smaller scope that they have access to. Any `SELECT INTO` statements SHALL succeed\n' + 'when done by a user with privilege granted by a user with `GRANT OPTION`,\n' + 'either directly or through an assigned role.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support a user with **grant option** privilege\n' + 'revoking **select** privilege from other **users** or **roles** on the same\n' + 'or smaller scope that they have access to. Any `SELECT INTO` statements SHALL fail\n' + 'when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' + 'either directly or through an assigned role, unless they have access otherwise.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support changing access to the **insert** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `INSERT INTO` statements SHALL not to be executed, unless the user\n' + 'has the **insert** privilege for the destination table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **insert** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **insert** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_Column = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.Column', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' + 'for one or more specified columns in a table to one or more **users** or **roles**.\n' + 'Any `INSERT INTO` statements SHALL not to be executed, unless the user\n' + 'has the **insert** privilege for the destination column\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `INSERT INTO` statements SHALL succeed only on nodes where\n' + 'the table exists and privilege was granted.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_GrantOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.GrantOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **insert** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' + 'change access to the **insert** privilege by another user or role\n' + 'on the same or smaller scope that they have access to.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support a user with **grant option** privilege\n' + 'granting **insert** privilege to other **users** or **roles** on the same\n' + 'or smaller scope that they have access to. Any `INSERT INTO` statements SHALL succeed\n' + 'when done by a user with privilege granted by a user with `GRANT OPTION`,\n' + 'either directly or through an assigned role.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support a user with **grant option** privilege\n' + 'revoking **insert** privilege from other **users** or **roles** on the same\n' + 'or smaller scope that they have access to. Any `INSERT INTO` statements SHALL fail\n' + 'when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' + 'either directly or through an assigned role, unless they have access otherwise.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Delete = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Delete', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **delete** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Alter = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Create = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **create** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_Drop = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **drop** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_All = Requirement( + name='RQ.SRS-006.RBAC.Privileges.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL include in the **all** privilege the same rights\n' + 'as provided by **usage**, **select**, **select columns**,\n' + '**insert**, **delete**, **alter**, **create**, and **drop** privileges.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_All_GrantRevoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.All.GrantRevoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **all** privileges\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.GrantOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **grant option** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_Privileges_AdminOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AdminOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **admin option** privilege\n' + 'to one or more **users** or **roles**.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_Create = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow any `CREATE` statements\n' + 'to be executed unless the user has the **create** privilege for the destination database\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_Alter = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow any `ALTER` statements\n' + 'to be executed unless the user has the **alter** privilege for the destination table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_Drop = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow any `DROP` statements\n' + 'to be executed unless the user has the **drop** privilege for the destination database\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_Drop_Table = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow any `DROP TABLE` statements\n' + 'to be executed unless the user has the **drop** privilege for the destination database or the table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_GrantRevoke = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements\n' + 'to be executed unless the user has the **grant option** privilege\n' + 'for the privilege of the destination table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_Use = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.Use', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow the `USE` statement to be executed\n' + 'unless the user has at least one of the privileges for the database\n' + 'or the table inside that database\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + ), + link=None + ) + +RQ_SRS_006_RBAC_RequiredPrivileges_Admin = Requirement( + name='RQ.SRS-006.RBAC.RequiredPrivileges.Admin', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not allow any of the following statements\n' + '\n' + '* `SYSTEM`\n' + '* `SHOW`\n' + '* `ATTACH`\n' + '* `CHECK TABLE`\n' + '* `DESCRIBE TABLE`\n' + '* `DETACH`\n' + '* `EXISTS`\n' + '* `KILL QUERY`\n' + '* `KILL MUTATION`\n' + '* `OPTIMIZE`\n' + '* `RENAME`\n' + '* `TRUNCATE`\n' + '\n' + 'to be executed unless the user has the **admin option** privilege\n' + 'through one of the roles with **admin option** privilege assigned to the user.\n' + ), + link=None + ) diff --git a/tests/testflows/rbac/tests/errors.py b/tests/testflows/rbac/tests/errors.py index 749d92e047b..0ca927b54e4 100755 --- a/tests/testflows/rbac/tests/errors.py +++ b/tests/testflows/rbac/tests/errors.py @@ -84,3 +84,7 @@ def unknown_setting(setting): def cluster_not_found(cluster): return (170, f"Exception: Requested cluster '{cluster}' not found") +## Privileges + +def not_enough_privileges(name): + return (241, f"Exception: {name}: Not enough privileges.") diff --git a/tests/testflows/rbac/tests/privileges/__init__.py b/tests/testflows/rbac/tests/privileges/__init__.py new file mode 100755 index 00000000000..d5866796e29 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/__init__.py @@ -0,0 +1,7 @@ +# Copyright 2020, Altinity LTD. All Rights Reserved. +# +# All information contained herein is, and remains the property +# of Altinity LTD. Any dissemination of this information or +# reproduction of this material is strictly forbidden unless +# prior written permission is obtained from Altinity LTD. +# \ No newline at end of file diff --git a/tests/testflows/rbac/tests/privileges/feature.py b/tests/testflows/rbac/tests/privileges/feature.py new file mode 100755 index 00000000000..da86a5d8111 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/feature.py @@ -0,0 +1,6 @@ +from testflows.core import * + +@TestFeature +@Name("privileges") +def feature(self): + Feature(run=load("rbac.tests.privileges.select", "feature"), flags=TE) diff --git a/tests/testflows/rbac/tests/privileges/select.py b/tests/testflows/rbac/tests/privileges/select.py new file mode 100644 index 00000000000..3ff017d5d20 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/select.py @@ -0,0 +1,487 @@ +from contextlib import contextmanager +import json + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +import rbac.tests.errors as errors + +table_types = { + "MergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111)", + "CollapsingMergeTree": "CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = CollapsingMergeTree(d, (a, b), 111, y);" +} + +@contextmanager +def table(node, name, table_type="MergeTree"): + try: + with Given(f"I have a {table_type} table"): + node.query(table_types[table_type].format(name=name)) + yield + finally: + with Finally("I drop the table"): + node.query(f"DROP TABLE IF EXISTS {name}") + +@contextmanager +def user(node, name): + try: + with Given("I have a user"): + node.query(f"CREATE USER OR REPLACE {name}") + yield + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {name}") + +@contextmanager +def role(node, role): + try: + with Given("I have a role"): + node.query(f"CREATE ROLE OR REPLACE {role}") + yield + finally: + with Finally("I drop the role"): + node.query(f"DROP ROLE IF EXISTS {role}") + +@TestScenario +def without_privilege(self, table_type, node=None): + """Check that user without select privilege on a table is not able to select on that table. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"): + with When("I run SELECT without privilege"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("SELECT * FROM merge_tree", settings = [("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_Grant("1.0"), +) +def user_with_privilege(self, table_type, node=None): + """Check that user can select from a table on which they have select privilege. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with user(node, "user88"): + pass + with user(node, "user0"): + with When("I grant privilege"): + node.query("GRANT SELECT ON merge_tree TO user0") + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_Revoke("1.0"), +) +def user_with_revoked_privilege(self, table_type, node=None): + """Check that user is unable to select from a table after select privilege + on that table has been revoked from the user. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"): + with When("I grant privilege"): + node.query("GRANT SELECT ON merge_tree TO user0") + with And("I revoke privilege"): + node.query("REVOKE SELECT ON merge_tree FROM user0") + with And("I use SELECT, throws exception"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("SELECT * FROM merge_tree", settings = [("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +def user_with_privilege_on_columns(self, table_type): + Scenario(run=user_column_privileges, + examples=Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass table_type", + [tuple(list(row)+[table_type]) for row in user_column_privileges.examples])) + +@TestOutline(Scenario) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"), +) +@Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass", [ + ("d", "d", "x", "d", '\'2020-01-01\''), + ("d,a", "d", "x", "d", '\'2020-01-01\''), + ("d,a,b", "d,a,b", "x", "d,b", '\'2020-01-01\',9'), + ("d,a,b", "b", "y", "d,a,b", '\'2020-01-01\',\'woo\',9') +]) +def user_column_privileges(self, grant_columns, select_columns_pass, data_pass, table_type, revoke_columns=None, select_columns_fail=None, node=None): + """Check that user is able to select on granted columns + and unable to select on not granted or revoked columns. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type), user(node, "user0"): + with Given("The table has some data on some columns"): + node.query(f"INSERT INTO merge_tree ({select_columns_pass}) VALUES ({data_pass})") + with When("I grant select privilege"): + node.query(f"GRANT SELECT({grant_columns}) ON merge_tree TO user0") + if select_columns_fail is not None: + with And("I select from not granted column"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query(f"SELECT ({select_columns_fail}) FROM merge_tree", + settings = [("user","user0")], exitcode=exitcode, message=message) + with Then("I select from granted column, verify correct result"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output + if revoke_columns is not None: + with When("I revoke select privilege for columns from user"): + node.query(f"REVOKE SELECT({revoke_columns}) ON merge_tree FROM user0") + with And("I select from revoked columns"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query(f"SELECT ({select_columns_pass}) FROM merge_tree", settings = [("user","user0")], exitcode=exitcode, message=message) + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_Grant("1.0"), +) +def role_with_privilege(self, table_type, node=None): + """Check that user can select from a table after it is granted a role that + has the select privilege for that table. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with user(node, "user0"): + with role(node, "role0"): + with When("I grant select privilege to a role"): + node.query("GRANT SELECT ON merge_tree TO role0") + with And("I grant role to the user"): + node.query("GRANT role0 TO user0") + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_Revoke("1.0"), +) +def role_with_revoked_privilege(self, table_type, node=None): + """Check that user with a role that has select privilege on a table is unable + to select from that table after select privilege has been revoked from the role. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"), role(node, "role0"): + with When("I grant privilege to a role"): + node.query("GRANT SELECT ON merge_tree TO role0") + with And("I grant the role to a user"): + node.query("GRANT role0 TO user0") + with And("I revoke privilege from the role"): + node.query("REVOKE SELECT ON merge_tree FROM role0") + with And("I select from the table"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("SELECT * FROM merge_tree", settings = [("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +def user_with_revoked_role(self, table_type, node=None): + """Check that user with a role that has select privilege on a table is unable to + select from that table after the role with select privilege has been revoked from the user. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"), role(node, "role0"): + with When("I grant privilege to a role"): + node.query("GRANT SELECT ON merge_tree TO role0") + with And("I grant the role to a user"): + node.query("GRANT role0 TO user0") + with And("I revoke the role from the user"): + node.query("REVOKE role0 FROM user0") + with And("I select from the table"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("SELECT * FROM merge_tree", settings = [("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +def role_with_privilege_on_columns(self, table_type): + Scenario(run=role_column_privileges, + examples=Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass table_type", + [tuple(list(row)+[table_type]) for row in role_column_privileges.examples])) + +@TestOutline(Scenario) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"), +) +@Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass", [ + ("d", "d", "x", "d", '\'2020-01-01\''), + ("d,a", "d", "x", "d", '\'2020-01-01\''), + ("d,a,b", "d,a,b", "x", "d,b", '\'2020-01-01\',9'), + ("d,a,b", "b", "y", "d,a,b", '\'2020-01-01\',\'woo\',9') +]) +def role_column_privileges(self, grant_columns, select_columns_pass, data_pass, table_type, revoke_columns=None, select_columns_fail=None, node=None): + """Check that user is able to select from granted columns and unable + to select from not granted or revoked columns. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("The table has some data on some columns"): + node.query(f"INSERT INTO merge_tree ({select_columns_pass}) VALUES ({data_pass})") + with user(node, "user0"), role(node, "role0"): + with When("I grant select privilege"): + node.query(f"GRANT SELECT({grant_columns}) ON merge_tree TO role0") + with And("I grant the role to a user"): + node.query("GRANT role0 TO user0") + if select_columns_fail is not None: + with And("I select from not granted column"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query(f"SELECT ({select_columns_fail}) FROM merge_tree", + settings = [("user","user0")], exitcode=exitcode, message=message) + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + if revoke_columns is not None: + with When("I revoke select privilege for columns from role"): + node.query(f"REVOKE SELECT({revoke_columns}) ON merge_tree FROM role0") + with And("I select from revoked columns"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query(f"SELECT ({select_columns_pass}) FROM merge_tree", + settings = [("user","user0")], exitcode=exitcode, message=message) + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), +) +def user_with_privilege_on_cluster(self, table_type, node=None): + """Check that user is able to select from a table with + privilege granted on a cluster. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + try: + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with Given("I have a user on a cluster"): + node.query("CREATE USER OR REPLACE user0 ON CLUSTER sharded_cluster") + with When("I grant select privilege on a cluster"): + node.query("GRANT ON CLUSTER sharded_cluster SELECT ON merge_tree TO user0") + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + finally: + with Finally("I drop the user"): + node.query("DROP USER user0 ON CLUSTER sharded_cluster") + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), +) +def user_with_privilege_from_user_with_grant_option(self, table_type, node=None): + """Check that user is able to select from a table when granted privilege + from another user with grant option. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with user(node, "user0"), user(node, "user1"): + with When("I grant privilege with grant option to user"): + node.query("GRANT SELECT ON merge_tree TO user0 WITH GRANT OPTION") + with And("I grant privilege to another user via grant option"): + node.query("GRANT SELECT ON merge_tree TO user1", settings = [("user","user0")]) + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), +) +def role_with_privilege_from_user_with_grant_option(self, table_type, node=None): + """Check that user is able to select from a table when granted a role with + select privilege that was granted by another user with grant option. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with user(node, "user0"), user(node, "user1"), role(node, "role0"): + with When("I grant privilege with grant option to user"): + node.query("GRANT SELECT ON merge_tree TO user0 WITH GRANT OPTION") + with And("I grant privilege to a role via grant option"): + node.query("GRANT SELECT ON merge_tree TO role0", settings = [("user","user0")]) + with And("I grant the role to another user"): + node.query("GRANT role0 TO user1") + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), +) +def user_with_privilege_from_role_with_grant_option(self, table_type, node=None): + """Check that user is able to select from a table when granted privilege from + a role with grant option + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with user(node, "user0"), user(node, "user1"), role(node, "role0"): + with When("I grant privilege with grant option to a role"): + node.query("GRANT SELECT ON merge_tree TO role0 WITH GRANT OPTION") + with When("I grant role to a user"): + node.query("GRANT role0 TO user0") + with And("I grant privilege to a user via grant option"): + node.query("GRANT SELECT ON merge_tree TO user1", settings = [("user","user0")]) + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), +) +def role_with_privilege_from_role_with_grant_option(self, table_type, node=None): + """Check that a user is able to select from a table with a role that was + granted privilege by another role with grant option + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with Given("I have some data inserted into table"): + node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") + with user(node, "user0"), user(node, "user1"), role(node, "role0"), role(node, "role1"): + with When("I grant privilege with grant option to role"): + node.query("GRANT SELECT ON merge_tree TO role0 WITH GRANT OPTION") + with And("I grant the role to a user"): + node.query("GRANT role0 TO user0") + with And("I grant privilege to another role via grant option"): + node.query("GRANT SELECT ON merge_tree TO role1", settings = [("user","user0")]) + with And("I grant the second role to another user"): + node.query("GRANT role1 TO user1") + with Then("I verify SELECT command"): + user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) + default = node.query("SELECT d FROM merge_tree") + assert user_select.output == default.output, error() + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), +) +def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None): + """Check that user is unable to revoke a column they don't have access to from a user. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"), user(node, "user1"): + with When("I grant privilege with grant option to user"): + node.query("GRANT SELECT(d) ON merge_tree TO user0 WITH GRANT OPTION") + with Then("I revoke privilege on a column the user with grant option does not have access to"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("REVOKE SELECT(b) ON merge_tree FROM user1", settings=[("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), +) +def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None): + """Check that user is unable to revoke a column they dont have acces to from a role. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"), role(node, "role0"): + with When("I grant privilege with grant option to user"): + node.query("GRANT SELECT(d) ON merge_tree TO user0 WITH GRANT OPTION") + with Then("I revoke privilege on a column the user with grant option does not have access to"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("REVOKE SELECT(b) ON merge_tree FROM role0", settings=[("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), +) +def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None): + """Check that user with a role is unable to revoke a column they dont have acces to from a user. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"), user(node,"user1"), role(node, "role0"): + with When("I grant privilege with grant option to a role"): + node.query("GRANT SELECT(d) ON merge_tree TO role0 WITH GRANT OPTION") + with And("I grant the role to a user"): + node.query("GRANT role0 TO user0") + with Then("I revoke privilege on a column the user with grant option does not have access to"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("REVOKE SELECT(b) ON merge_tree FROM user1", settings=[("user","user0")], + exitcode=exitcode, message=message) + +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), +) +def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None): + """Check that user with a role is unable to revoke a column they dont have acces to from a role. + """ + if node is None: + node = self.context.node + with table(node, "merge_tree", table_type): + with user(node, "user0"), role(node, "role0"), role(node, "role1"): + with When("I grant privilege with grant option to a role"): + node.query("GRANT SELECT(d) ON merge_tree TO user0 WITH GRANT OPTION") + with And("I grant the role to a user"): + node.query("GRANT role0 TO user0") + with Then("I revoke privilege on a column the user with grant option does not have access to"): + exitcode, message = errors.not_enough_privileges(name="user0") + node.query("REVOKE SELECT(b) ON merge_tree FROM role1", settings=[("user","user0")], + exitcode=exitcode, message=message) + +@TestOutline(Feature) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Select("1.0"), +) +@Examples("table_type", [ + (key,) for key in table_types.keys() +]) +@Name("select") +def feature(self, table_type, node="clickhouse1"): + self.context.node = self.context.cluster.node(node) + + Scenario(test=without_privilege)(table_type=table_type) + Scenario(test=user_with_privilege)(table_type=table_type) + Scenario(test=user_with_revoked_privilege)(table_type=table_type) + Scenario(test=user_with_privilege_on_columns)(table_type=table_type) + Scenario(test=role_with_privilege)(table_type=table_type) + Scenario(test=role_with_revoked_privilege)(table_type=table_type) + Scenario(test=user_with_revoked_role)(table_type=table_type) + Scenario(test=role_with_privilege_on_columns)(table_type=table_type) + Scenario(test=user_with_privilege_on_cluster)(table_type=table_type) + Scenario(test=user_with_privilege_from_user_with_grant_option)(table_type=table_type) + Scenario(test=role_with_privilege_from_user_with_grant_option)(table_type=table_type) + Scenario(test=user_with_privilege_from_role_with_grant_option)(table_type=table_type) + Scenario(test=role_with_privilege_from_role_with_grant_option)(table_type=table_type) + Scenario(test=revoke_privilege_from_user_via_user_with_grant_option)(table_type=table_type) + Scenario(test=revoke_privilege_from_role_via_user_with_grant_option)(table_type=table_type) + Scenario(test=revoke_privilege_from_user_via_role_with_grant_option)(table_type=table_type) + Scenario(test=revoke_privilege_from_role_via_role_with_grant_option)(table_type=table_type) \ No newline at end of file diff --git a/tests/testflows/rbac/tests/syntax/__init__.py b/tests/testflows/rbac/tests/syntax/__init__.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/alter_quota.py b/tests/testflows/rbac/tests/syntax/alter_quota.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/alter_role.py b/tests/testflows/rbac/tests/syntax/alter_role.py old mode 100644 new mode 100755 index 839fe57c8df..b8e1de387c9 --- a/tests/testflows/rbac/tests/syntax/alter_role.py +++ b/tests/testflows/rbac/tests/syntax/alter_role.py @@ -33,7 +33,7 @@ def feature(self, node="clickhouse1"): if profile != "": with Finally("I drop the profile"): node.query(f"DROP SETTINGS PROFILE IF EXISTS {profile}") - + def cleanup_role(role): with Given(f"I ensure that role {role} does not exist"): node.query(f"DROP ROLE IF EXISTS {role}") @@ -66,7 +66,7 @@ def feature(self, node="clickhouse1"): with When(f"I alter role {role} that does not exist"): node.query(f"ALTER ROLE IF EXISTS {role}") del role - + with Scenario("I alter role on cluster", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Alter_Cluster("1.0")]): try: @@ -101,7 +101,7 @@ def feature(self, node="clickhouse1"): with Finally(f"I cleanup new name {new_role}"): node.query(f"DROP ROLE IF EXISTS {new_role}") del new_role - + with Scenario("I alter role to rename, new name is not available, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Alter_Rename("1.0")]): with setup("role2a"): @@ -131,7 +131,7 @@ def feature(self, node="clickhouse1"): with When("I alter role with settings profile that does not exist"): exitcode, message = errors.settings_profile_not_found_in_disk("profile0") node.query("ALTER ROLE role4a SETTINGS PROFILE profile0", exitcode=exitcode, message=message) - + with Scenario("I alter role settings profile multiple", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Alter_Settings("1.0")]): with setup("role4b", profile="profile0"): @@ -144,7 +144,7 @@ def feature(self, node="clickhouse1"): with setup("role5"): with When("I alter role with settings and no profile"): node.query("ALTER ROLE role5 SETTINGS max_memory_usage=10000000 READONLY") - + with Scenario("I alter role settings, variable does not exist, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Alter_Settings("1.0")]): with setup("role5a"): @@ -181,12 +181,12 @@ def feature(self, node="clickhouse1"): with Scenario("I alter role settings min, with and without = sign", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Alter_Settings("1.0")]): - with setup("role10"): + with setup("role10"): with When("I set min, no equals"): node.query("ALTER ROLE role10 SETTINGS max_memory_usage MIN 200") with When("I set min, yes equals"): node.query("ALTER ROLE role10 SETTINGS max_memory_usage MIN = 200") - + with Scenario("I alter role settings max, with and without = sign", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Alter_Settings("1.0")]): with setup("role11"): diff --git a/tests/testflows/rbac/tests/syntax/alter_row_policy.py b/tests/testflows/rbac/tests/syntax/alter_row_policy.py old mode 100644 new mode 100755 index 79af04db771..969415b200d --- a/tests/testflows/rbac/tests/syntax/alter_row_policy.py +++ b/tests/testflows/rbac/tests/syntax/alter_row_policy.py @@ -31,7 +31,7 @@ def feature(self, node="clickhouse1"): finally: with Finally("I drop the row policy"): node.query(f"DROP ROW POLICY IF EXISTS {policy} ON default.foo") - + def cleanup_policy(policy): with Given(f"I ensure that policy {policy} does not exist"): node.query(f"DROP ROW POLICY IF EXISTS {policy} ON default.foo") @@ -55,7 +55,7 @@ def feature(self, node="clickhouse1"): with cleanup("policy1"): with When("I alter row policy short form"): node.query("ALTER POLICY policy1 ON default.foo") - + with Scenario("I alter row policy, does not exist, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_RowPolicy_Alter("1.0"), RQ_SRS_006_RBAC_RowPolicy_Alter_On("1.0")]): @@ -72,7 +72,7 @@ def feature(self, node="clickhouse1"): with cleanup("policy2"): with When("I alter row policy using if exists"): node.query("ALTER ROW POLICY IF EXISTS policy2 ON default.foo") - + with Scenario("I alter row policy if exists, policy does not exist", flags=TE, requirements=[ RQ_SRS_006_RBAC_RowPolicy_Alter_IfExists("1.0"), RQ_SRS_006_RBAC_RowPolicy_Alter_On("1.0")]): @@ -203,7 +203,7 @@ def feature(self, node="clickhouse1"): RQ_SRS_006_RBAC_RowPolicy_Alter_On("1.0")]): with cleanup("policy12"): with When("I alter row policy to no assignment"): - node.query("ALTER ROW POLICY policy12 ON default.foo TO NONE") + node.query("ALTER ROW POLICY policy12 ON default.foo TO NONE") # Official syntax: ON CLUSTER cluster_name ON database.table # Working syntax: both orderings of ON CLUSTER and TABLE clauses work diff --git a/tests/testflows/rbac/tests/syntax/alter_settings_profile.py b/tests/testflows/rbac/tests/syntax/alter_settings_profile.py old mode 100644 new mode 100755 index d4ce65da785..9b1f06be48b --- a/tests/testflows/rbac/tests/syntax/alter_settings_profile.py +++ b/tests/testflows/rbac/tests/syntax/alter_settings_profile.py @@ -34,7 +34,7 @@ def feature(self, node="clickhouse1"): with Scenario("I alter settings profile with no options", flags=TE, requirements=[RQ_SRS_006_RBAC_SettingsProfile_Alter("1.0")]): with When("I alter settings profile"): node.query("ALTER SETTINGS PROFILE profile0") - + with Scenario("I alter settings profile short form", flags=TE, requirements=[RQ_SRS_006_RBAC_SettingsProfile_Alter("1.0")]): with When("I short form alter settings profile"): node.query("ALTER PROFILE profile0") @@ -58,7 +58,7 @@ def feature(self, node="clickhouse1"): cleanup_profile(profile) with When(f"I alter settings profile {profile} using if exists"): node.query(f"ALTER SETTINGS PROFILE IF EXISTS {profile}") - + del profile with Scenario("I alter settings profile to rename, target available", flags=TE, requirements=[RQ_SRS_006_RBAC_SettingsProfile_Alter_Rename("1.0")]): @@ -78,7 +78,7 @@ def feature(self, node="clickhouse1"): finally: with Finally(f"I cleanup target name {new_profile}"): node.query(f"DROP SETTINGS PROFILE IF EXISTS {new_profile}") - + del new_profile with Scenario("I alter settings profile with a setting value", flags=TE, requirements=[ diff --git a/tests/testflows/rbac/tests/syntax/alter_user.py b/tests/testflows/rbac/tests/syntax/alter_user.py old mode 100644 new mode 100755 index aab610c8f79..1fd8e62239a --- a/tests/testflows/rbac/tests/syntax/alter_user.py +++ b/tests/testflows/rbac/tests/syntax/alter_user.py @@ -80,7 +80,7 @@ def feature(self, node="clickhouse1"): with setup("user15"): with When("I alter user name"): node.query("ALTER USER user15 RENAME TO user15") - + with Scenario("I alter user to rename, target unavailable", flags=TE, requirements=[ RQ_SRS_006_RBAC_User_Alter_Rename("1.0")]): with setup("user15"): @@ -174,7 +174,7 @@ def feature(self, node="clickhouse1"): node.query("ALTER USER user13 DEFAULT ROLE ALL") @contextmanager - def setup_role(role): + def setup_role(role): try: with Given(f"I have a role {role}"): node.query(f"CREATE ROLE OR REPLACE {role}") @@ -198,7 +198,7 @@ def feature(self, node="clickhouse1"): node.query("GRANT default TO user14a") with When("I alter user default role"): node.query("ALTER USER user14a DEFAULT ROLE default") - + with Scenario("I alter user default role, role doesn't exist, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_User_Alter_DefaultRole("1.0")]): with setup("user12"): @@ -228,7 +228,7 @@ def feature(self, node="clickhouse1"): node.query("GRANT second,third TO user15") with When("I alter user default role to second, third"): node.query("ALTER USER user15 DEFAULT ROLE second, third") - + with Scenario("I alter user default role set to all except", flags=TE, requirements=[ RQ_SRS_006_RBAC_User_Alter_DefaultRole_AllExcept("1.0")]): with setup("user16"), setup_role("second"): diff --git a/tests/testflows/rbac/tests/syntax/create_quota.py b/tests/testflows/rbac/tests/syntax/create_quota.py old mode 100644 new mode 100755 index 9697da2ac29..51e19b7f793 --- a/tests/testflows/rbac/tests/syntax/create_quota.py +++ b/tests/testflows/rbac/tests/syntax/create_quota.py @@ -31,11 +31,11 @@ def feature(self, node="clickhouse1"): finally: with Finally("I drop the quota"): node.query(f"DROP QUOTA IF EXISTS {quota}") - + def create_quota(quota): with And(f"I ensure I do have quota {quota}"): node.query(f"CREATE QUOTA OR REPLACE {quota}") - + try: with Given("I have a user and a role"): node.query(f"CREATE USER user0") @@ -81,7 +81,7 @@ def feature(self, node="clickhouse1"): with When(f"I create a quota {quota} with or replace"): node.query(f"CREATE QUOTA OR REPLACE {quota}") del quota - + with Scenario("I create quota or replace, quota does exist", flags=TE, requirements=[ RQ_SRS_006_RBAC_Quota_Create_Replace("1.0")]): quota = "quota2" @@ -214,7 +214,7 @@ def feature(self, node="clickhouse1"): finally: with Finally("I drop the quota from cluster"): node.query("DROP QUOTA IF EXISTS quota29 ON CLUSTER sharded_cluster") - + with Scenario("I create quota on nonexistent cluster, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_Quota_Create_Cluster("1.0")]): with When("I run create quota on a cluster"): diff --git a/tests/testflows/rbac/tests/syntax/create_role.py b/tests/testflows/rbac/tests/syntax/create_role.py old mode 100644 new mode 100755 index f87710f992b..0e42c84ef06 --- a/tests/testflows/rbac/tests/syntax/create_role.py +++ b/tests/testflows/rbac/tests/syntax/create_role.py @@ -54,7 +54,7 @@ def feature(self, node="clickhouse1"): with When(f"I create role {role} with if not exists"): node.query(f"CREATE ROLE IF NOT EXISTS {role}") del role - + with Scenario("I create role if not exists, role does exist", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Create_IfNotExists("1.0")]): role = "role1" diff --git a/tests/testflows/rbac/tests/syntax/create_row_policy.py b/tests/testflows/rbac/tests/syntax/create_row_policy.py old mode 100644 new mode 100755 index 458b205e6c1..f733c7ea505 --- a/tests/testflows/rbac/tests/syntax/create_row_policy.py +++ b/tests/testflows/rbac/tests/syntax/create_row_policy.py @@ -30,7 +30,7 @@ def feature(self, node="clickhouse1"): finally: with Finally(f"I drop the row policy on {on}"): node.query(f"DROP ROW POLICY IF EXISTS {policy} ON {on}") - + def create_policy(policy, on="default.foo"): with Given(f"I ensure I do have policy {policy} on {on}"): node.query(f"CREATE ROW POLICY OR REPLACE {policy} ON {on}") @@ -47,7 +47,7 @@ def feature(self, node="clickhouse1"): with cleanup("policy0"): with When("I create row policy"): node.query("CREATE ROW POLICY policy0 ON default.foo") - + with Scenario("I create row policy using short syntax with no options", flags=TE, requirements=[ RQ_SRS_006_RBAC_RowPolicy_Create("1.0"), RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]): @@ -89,7 +89,7 @@ def feature(self, node="clickhouse1"): with cleanup("policy3"): with When("I create row policy with or replace"): node.query("CREATE ROW POLICY OR REPLACE policy3 ON default.foo") - + with Scenario("I create row policy or replace, policy does exist", flags=TE, requirements=[ RQ_SRS_006_RBAC_RowPolicy_Create_Replace("1.0"), RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]): diff --git a/tests/testflows/rbac/tests/syntax/create_settings_profile.py b/tests/testflows/rbac/tests/syntax/create_settings_profile.py old mode 100644 new mode 100755 index 1cd5289db26..d4987ba4d8d --- a/tests/testflows/rbac/tests/syntax/create_settings_profile.py +++ b/tests/testflows/rbac/tests/syntax/create_settings_profile.py @@ -44,7 +44,7 @@ def feature(self, node="clickhouse1"): with cleanup("profile0"): with When("I create settings profile"): node.query("CREATE SETTINGS PROFILE profile0") - + with Scenario("I create settings profile that already exists, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_SettingsProfile_Create("1.0")]): profile = "profile0" @@ -54,7 +54,7 @@ def feature(self, node="clickhouse1"): exitcode, message = errors.cannot_insert_settings_profile(name=profile) node.query(f"CREATE SETTINGS PROFILE {profile}", exitcode=exitcode, message=message) del profile - + with Scenario("I create settings profile if not exists, profile does not exist", flags=TE, requirements=[ RQ_SRS_006_RBAC_SettingsProfile_Create_IfNotExists("1.0")]): with cleanup("profile1"): diff --git a/tests/testflows/rbac/tests/syntax/create_user.py b/tests/testflows/rbac/tests/syntax/create_user.py old mode 100644 new mode 100755 index 874421f2c81..6b234f857c9 --- a/tests/testflows/rbac/tests/syntax/create_user.py +++ b/tests/testflows/rbac/tests/syntax/create_user.py @@ -190,7 +190,7 @@ def feature(self, node="clickhouse1"): with Scenario("I create user with default role", flags=TE, requirements=[ RQ_SRS_006_RBAC_User_Create_DefaultRole("1.0")]): with Given("I have a role"): - node.query("CREATE ROLE default") + node.query("CREATE ROLE default") with cleanup("user9"): with When("I create a user with a default role"): node.query("CREATE USER user9 DEFAULT ROLE default") diff --git a/tests/testflows/rbac/tests/syntax/drop_quota.py b/tests/testflows/rbac/tests/syntax/drop_quota.py old mode 100644 new mode 100755 index a90294800c2..510ed714c64 --- a/tests/testflows/rbac/tests/syntax/drop_quota.py +++ b/tests/testflows/rbac/tests/syntax/drop_quota.py @@ -25,7 +25,7 @@ def feature(self, node="clickhouse1"): finally: with Finally("I drop the quota"): node.query(f"DROP QUOTA IF EXISTS {quota}") - + def cleanup_quota(quota): with Given(f"I ensure that quota {quota} does not exist"): node.query(f"DROP QUOTA IF EXISTS {quota}") diff --git a/tests/testflows/rbac/tests/syntax/drop_role.py b/tests/testflows/rbac/tests/syntax/drop_role.py old mode 100644 new mode 100755 index e3f89298a50..76fd50c1a39 --- a/tests/testflows/rbac/tests/syntax/drop_role.py +++ b/tests/testflows/rbac/tests/syntax/drop_role.py @@ -25,7 +25,7 @@ def feature(self, node="clickhouse1"): finally: with Finally("I confirm the role is dropped"): node.query(f"DROP ROLE IF EXISTS {role}") - + def cleanup_role(role): with Given(f"I ensure that role {role} does not exist"): node.query(f"DROP ROLE IF EXISTS {role}") @@ -69,7 +69,7 @@ def feature(self, node="clickhouse1"): node.query("DROP ROLE IF EXISTS role6") with When("I drop the nonexistant roles"): node.query("DROP USER IF EXISTS role5, role6") - + with Scenario("I drop role on cluster", flags=TE, requirements=[ RQ_SRS_006_RBAC_Role_Drop_Cluster("1.0")]): with Given("I have a role on cluster"): diff --git a/tests/testflows/rbac/tests/syntax/drop_row_policy.py b/tests/testflows/rbac/tests/syntax/drop_row_policy.py old mode 100644 new mode 100755 index 37831ea9d27..218b088e443 --- a/tests/testflows/rbac/tests/syntax/drop_row_policy.py +++ b/tests/testflows/rbac/tests/syntax/drop_row_policy.py @@ -29,7 +29,7 @@ def feature(self, node="clickhouse1"): for i in policy: for j in on: node.query(f"DROP ROW POLICY IF EXISTS {i} ON {j}") - + def cleanup_policy(policy, on="default.foo"): with Given(f"I ensure that policy {policy} does not exist"): node.query(f"DROP ROW POLICY IF EXISTS {policy} ON {on}") @@ -40,21 +40,21 @@ def feature(self, node="clickhouse1"): node.query(f"CREATE TABLE default.foo2 (x UInt64, y String) Engine=Memory") with Scenario("I drop row policy with no options", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with cleanup(["policy1"]): with When("I drop row policy"): node.query("DROP ROW POLICY policy1 ON default.foo") with Scenario("I drop row policy using short syntax with no options", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with cleanup(["policy2"]): with When("I drop row policy short form"): node.query("DROP POLICY policy2 ON default.foo") with Scenario("I drop row policy, does not exist, throws exception", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): policy = "policy1" cleanup_policy(policy) @@ -64,42 +64,42 @@ def feature(self, node="clickhouse1"): del policy with Scenario("I drop row policy if exists, policy does exist", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with cleanup(["policy3"]): with When("I drop row policy if exists"): node.query("DROP ROW POLICY IF EXISTS policy3 ON default.foo") with Scenario("I drop row policy if exists, policy doesn't exist", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): cleanup_policy("policy3") with When("I drop row policy if exists"): node.query("DROP ROW POLICY IF EXISTS policy3 ON default.foo") with Scenario("I drop multiple row policies", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with cleanup(["policy3", "policy4"]): with When("I drop multiple row policies"): node.query("DROP ROW POLICY policy3, policy4 ON default.foo") with Scenario("I drop row policy on multiple tables", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with cleanup(["policy3"], ["default.foo","default.foo2"]): with When("I drop row policy on multiple tables"): node.query("DROP ROW POLICY policy3 ON default.foo, default.foo2") with Scenario("I drop multiple row policies on multiple tables", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with cleanup(["policy3", "policy4"], ["default.foo","default.foo2"]): with When("I drop the row policies from the tables"): node.query("DROP ROW POLICY policy3 ON default.foo, policy4 ON default.foo2") with Scenario("I drop row policy on cluster", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): try: with Given("I have a row policy"): @@ -111,7 +111,7 @@ def feature(self, node="clickhouse1"): node.query("DROP ROW POLICY IF EXISTS policy13 ON default.foo ON CLUSTER sharded_cluster") with Scenario("I drop row policy on cluster after table", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): try: with Given("I have a row policy"): @@ -123,7 +123,7 @@ def feature(self, node="clickhouse1"): node.query("DROP ROW POLICY IF EXISTS policy12 ON default.foo ON CLUSTER sharded_cluster") with Scenario("I drop row policy on fake cluster throws exception", flags=TE, requirements=[ - RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster("1.0"), + RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster("1.0"), RQ_SRS_006_RBAC_RowPolicy_Drop_On("1.0")]): with When("I run drop row policy command"): exitcode, message = errors.cluster_not_found("fake_cluster") diff --git a/tests/testflows/rbac/tests/syntax/drop_settings_profile.py b/tests/testflows/rbac/tests/syntax/drop_settings_profile.py old mode 100644 new mode 100755 index aea5194c9c9..b375a51a67d --- a/tests/testflows/rbac/tests/syntax/drop_settings_profile.py +++ b/tests/testflows/rbac/tests/syntax/drop_settings_profile.py @@ -25,7 +25,7 @@ def feature(self, node="clickhouse1"): finally: with Finally("I drop the settings profile"): node.query(f"DROP SETTINGS PROFILE IF EXISTS {profile}") - + def cleanup_profile(profile): with Given(f"I ensure that profile {profile} does not exist"): node.query(f"DROP SETTINGS PROFILE IF EXISTS {profile}") diff --git a/tests/testflows/rbac/tests/syntax/drop_user.py b/tests/testflows/rbac/tests/syntax/drop_user.py old mode 100644 new mode 100755 index c5a2d16d7e1..beeb4af31e6 --- a/tests/testflows/rbac/tests/syntax/drop_user.py +++ b/tests/testflows/rbac/tests/syntax/drop_user.py @@ -35,7 +35,7 @@ def feature(self, node="clickhouse1"): with setup("user0"): with When("I drop user"): node.query("DROP USER user0") - + with Scenario("I drop user, does not exist, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_User_Drop("1.0")]): user = "user0" @@ -79,7 +79,7 @@ def feature(self, node="clickhouse1"): RQ_SRS_006_RBAC_User_Drop_IfExists("1.0")]): with When("I drop the nonexistant users"): node.query("DROP USER IF EXISTS user5, user6") - + with Scenario("I drop user from specific cluster", flags=TE, requirements=[ RQ_SRS_006_RBAC_User_Drop_OnCluster("1.0")]): try: diff --git a/tests/testflows/rbac/tests/syntax/feature.py b/tests/testflows/rbac/tests/syntax/feature.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/grant_privilege.py b/tests/testflows/rbac/tests/syntax/grant_privilege.py old mode 100644 new mode 100755 index fdb3224de8e..cabb3a3780b --- a/tests/testflows/rbac/tests/syntax/grant_privilege.py +++ b/tests/testflows/rbac/tests/syntax/grant_privilege.py @@ -19,7 +19,6 @@ def setup(node): node.query("DROP USER IF EXISTS user1") node.query("DROP ROLE IF EXISTS role1") - @TestOutline(Scenario) @Examples("privilege on allow_introspection", [ ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Grant_Privilege_DictGet("1.0"))), @@ -41,10 +40,10 @@ def setup(node): ],) def grant_privileges(self, privilege, on, allow_introspection, node="clickhouse1"): grant_privilege(privilege=privilege, on=on, allow_introspection=allow_introspection, node=node) - + @TestOutline(Scenario) @Requirements(RQ_SRS_006_RBAC_Grant_Privilege_GrantOption("1.0")) -def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): +def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): node = self.context.cluster.node(node) for on_ in on: @@ -62,7 +61,7 @@ def grant_privilege(self, privilege, on, allow_introspection, node="clickhouse1" #grant column specific for some column 'x' with When("I grant privilege with columns"): node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings) - + @TestFeature @Name("grant privilege") @Args(format_description=False) @@ -78,7 +77,7 @@ def feature(self, node="clickhouse1"): ``` """ node = self.context.cluster.node(node) - + Scenario(run=grant_privileges) # with nonexistant object name, GRANT assumes type role @@ -89,45 +88,45 @@ def feature(self, node="clickhouse1"): with When("I grant privilege ON CLUSTER"): exitcode, message = errors.role_not_found_in_disk(name="role0") node.query("GRANT NONE TO role0", exitcode=exitcode, message=message) - + with Scenario("I grant privilege ON CLUSTER", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Grant_Privilege_OnCluster("1.0"), + RQ_SRS_006_RBAC_Grant_Privilege_OnCluster("1.0"), RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]): with setup(node): with When("I grant privilege ON CLUSTER"): node.query("GRANT ON CLUSTER sharded_cluster NONE TO user0") - + with Scenario("I grant privilege on fake cluster, throws exception", flags=TE, requirements=[ RQ_SRS_006_RBAC_Grant_Privilege_OnCluster("1.0")]): with setup(node): with When("I grant privilege ON CLUSTER"): exitcode, message = errors.cluster_not_found("fake_cluster") node.query("GRANT ON CLUSTER fake_cluster NONE TO user0", exitcode=exitcode, message=message) - + with Scenario("I grant privilege to multiple users and roles", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Grant_Privilege_To("1.0"), + RQ_SRS_006_RBAC_Grant_Privilege_To("1.0"), RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]): with setup(node): with When("I grant privilege to several users"): node.query("GRANT NONE TO user0, user1, role1") - + with Scenario("I grant privilege to current user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser("1.0"), + RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser("1.0"), RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]): with setup(node): with When("I grant privilege to current user"): node.query("GRANT NONE TO CURRENT_USER", settings = [("user","user0")]) with Scenario("I grant privilege NONE to default user, throws exception", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser("1.0"), + RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser("1.0"), RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]): with setup(node): with When("I grant privilege to current user"): exitcode, message = errors.cannot_update_default() node.query("GRANT NONE TO CURRENT_USER", exitcode=exitcode, message=message) - + with Scenario("I grant privilege with grant option", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Grant_Privilege_GrantOption("1.0"), + RQ_SRS_006_RBAC_Grant_Privilege_GrantOption("1.0"), RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]): with setup(node): with When("I grant privilege with grant option"): diff --git a/tests/testflows/rbac/tests/syntax/grant_role.py b/tests/testflows/rbac/tests/syntax/grant_role.py old mode 100644 new mode 100755 index 48e5101e6cf..188d4141a3a --- a/tests/testflows/rbac/tests/syntax/grant_role.py +++ b/tests/testflows/rbac/tests/syntax/grant_role.py @@ -10,7 +10,7 @@ import rbac.tests.errors as errors @Args(format_description=False) def feature(self, node="clickhouse1"): """Check grant query syntax. - + ```sql GRANT ON CLUSTER [cluster_name] role [,...] TO {user | another_role | CURRENT_USER} [,...] [WITH ADMIN OPTION] ``` @@ -93,7 +93,7 @@ def feature(self, node="clickhouse1"): with setup(1,1): with When("I grant role to a user with admin option"): node.query("GRANT role0 TO user0 WITH ADMIN OPTION") - + with Scenario("I grant role to user on cluster", flags=TE, requirements=[ RQ_SRS_006_RBAC_Grant_Role_OnCluster("1.0")]): try: diff --git a/tests/testflows/rbac/tests/syntax/revoke_privilege.py b/tests/testflows/rbac/tests/syntax/revoke_privilege.py old mode 100644 new mode 100755 index 62130c4bf9a..0c9bf954ad8 --- a/tests/testflows/rbac/tests/syntax/revoke_privilege.py +++ b/tests/testflows/rbac/tests/syntax/revoke_privilege.py @@ -19,7 +19,6 @@ def setup(node): node.query("DROP USER IF EXISTS user1") node.query("DROP ROLE IF EXISTS role1") - @TestOutline(Scenario) @Examples("privilege on allow_introspection", [ ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_DictGet("1.0"))), @@ -41,11 +40,11 @@ def setup(node): ],) def revoke_privileges(self, privilege, on, allow_introspection, node="clickhouse1"): revoke_privilege(privilege=privilege, on=on, allow_introspection=allow_introspection, node=node) - + @TestOutline(Scenario) @Requirements([RQ_SRS_006_RBAC_Revoke_Privilege_Any("1.0") , RQ_SRS_006_RBAC_Revoke_Privilege_PrivelegeColumns("1.0")]) -def revoke_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): - node = self.context.cluster.node(node) +def revoke_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): + node = self.context.cluster.node(node) for on_ in on: with When(f"I revoke {privilege} privilege from user on {on_}"): with setup(node): @@ -59,7 +58,7 @@ def revoke_privilege(self, privilege, on, allow_introspection, node="clickhouse1 #revoke column specific for some column 'x' with When("I revoke privilege with columns"): node.query(f"REVOKE {privilege}(x) ON {on_} FROM user0", settings=settings) - + @TestFeature @Name("revoke privilege") @Args(format_description=False) @@ -69,23 +68,23 @@ def feature(self, node="clickhouse1"): ```sql REVOKE [ON CLUSTER cluster_name] privilege [(column_name [,...])] [,...] - ON {db.table|db.*|*.*|table|*} + ON {db.table|db.*|*.*|table|*} FROM {user | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user | CURRENT_USER} [,...] ``` """ node = self.context.cluster.node(node) - + Scenario(run=revoke_privileges) - + with Scenario("I revoke privilege ON CLUSTER", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_Cluster("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_Cluster("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege ON CLUSTER"): node.query("REVOKE ON CLUSTER sharded_cluster NONE FROM user0") - + with Scenario("I revoke privilege ON fake CLUSTER, throws exception", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_Cluster("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_Cluster("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege ON CLUSTER"): @@ -94,21 +93,21 @@ def feature(self, node="clickhouse1"): exitcode=exitcode, message=message) with Scenario("I revoke privilege from multiple users and roles", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege from multiple users"): node.query("REVOKE NONE FROM user0, user1, role1") - + with Scenario("I revoke privilege from current user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege from current user"): node.query("REVOKE NONE FROM CURRENT_USER", settings = [("user","user0")]) with Scenario("I revoke privilege from all users", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege from all users"): @@ -116,16 +115,16 @@ def feature(self, node="clickhouse1"): node.query("REVOKE NONE FROM ALL", exitcode=exitcode,message=message) with Scenario("I revoke privilege from default user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege from default user"): exitcode, message = errors.cannot_update_default() node.query("REVOKE NONE FROM default", exitcode=exitcode,message=message) - #By default, ClickHouse treats unnamed object as role + # By default, ClickHouse treats unnamed object as role with Scenario("I revoke privilege from nonexistent role, throws exception", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): role = "role5" with Given(f"I ensure that role {role} does not exist"): @@ -135,7 +134,7 @@ def feature(self, node="clickhouse1"): node.query(f"REVOKE NONE FROM {role}", exitcode=exitcode,message=message) with Scenario("I revoke privilege from ALL EXCEPT nonexistent role, throws exception", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): role = "role5" with Given(f"I ensure that role {role} does not exist"): @@ -145,14 +144,14 @@ def feature(self, node="clickhouse1"): node.query(f"REVOKE NONE FROM ALL EXCEPT {role}", exitcode=exitcode,message=message) with Scenario("I revoke privilege from all except some users and roles", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege all except some users"): node.query("REVOKE NONE FROM ALL EXCEPT default, user0, role1") with Scenario("I revoke privilege from all except current user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), + RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): with setup(node): with When("I revoke privilege from all except current user"): diff --git a/tests/testflows/rbac/tests/syntax/revoke_role.py b/tests/testflows/rbac/tests/syntax/revoke_role.py old mode 100644 new mode 100755 index 8b150c74705..d22083a86dc --- a/tests/testflows/rbac/tests/syntax/revoke_role.py +++ b/tests/testflows/rbac/tests/syntax/revoke_role.py @@ -50,7 +50,7 @@ def feature(self, node="clickhouse1"): exitcode, message = errors.role_not_found_in_disk(name="role0") node.query("REVOKE role0 FROM user0", exitcode=exitcode, message=message) - # with nonexistent object name, REVOKE assumes type role (treats user0 as role) + # With nonexistent object name, REVOKE assumes type role (treats user0 as role) with Scenario("I revoke a role from a nonexistent user", requirements=[ RQ_SRS_006_RBAC_Revoke_Role("1.0")]): with setup(0,1): @@ -58,7 +58,7 @@ def feature(self, node="clickhouse1"): exitcode, message = errors.role_not_found_in_disk(name="user0") node.query("REVOKE role0 FROM user0", exitcode=exitcode, message=message) - # with nonexistent object name, REVOKE assumes type role (treats user0 as role) + # With nonexistent object name, REVOKE assumes type role (treats user0 as role) with Scenario("I revoke a role from ALL EXCEPT nonexistent user", requirements=[ RQ_SRS_006_RBAC_Revoke_Role("1.0")]): with setup(0,1): @@ -86,7 +86,7 @@ def feature(self, node="clickhouse1"): #user is default, expect exception with Scenario("I revoke a role from default user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke a role from default user"): @@ -95,7 +95,7 @@ def feature(self, node="clickhouse1"): #user is user0 with Scenario("I revoke a role from current user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke a role from current user"): @@ -103,16 +103,16 @@ def feature(self, node="clickhouse1"): #user is default, expect exception with Scenario("I revoke a role from all", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke a role from all"): - exitcode, message = errors.cannot_update_default() + exitcode, message = errors.cannot_update_default() node.query("REVOKE role0 FROM ALL", exitcode=exitcode, message=message) #user is default, expect exception with Scenario("I revoke multiple roles from all", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke multiple roles from all"): @@ -120,14 +120,14 @@ def feature(self, node="clickhouse1"): node.query("REVOKE role0, role1 FROM ALL", exitcode=exitcode, message=message) with Scenario("I revoke a role from all but current user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke a role from all except current"): node.query("REVOKE role0 FROM ALL EXCEPT CURRENT_USER") with Scenario("I revoke a role from all but default user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke a role from all except default"): @@ -135,7 +135,7 @@ def feature(self, node="clickhouse1"): settings = [("user","user0")]) with Scenario("I revoke multiple roles from all but default user", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Keywords("1.0")]): with setup(): with When("I revoke multiple roles from all except default"): @@ -152,7 +152,7 @@ def feature(self, node="clickhouse1"): with setup(): with When("I revoke a role from multiple roles"): node.query("REVOKE role0 FROM role1, user0") - + with Scenario("I revoke a role from a user on cluster", flags=TE, requirements=[ RQ_SRS_006_RBAC_Revoke_Role_Cluster("1.0")]): with Given("I have a role and a user on a cluster"): @@ -171,7 +171,7 @@ def feature(self, node="clickhouse1"): node.query("REVOKE ON CLUSTER fake_cluster role0 FROM user0", exitcode=exitcode, message=message) with Scenario("I revoke multiple roles from multiple users on cluster", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_Role_Cluster("1.0")]): with Given("I have multiple roles and multiple users on a cluster"): for i in range(2): @@ -191,7 +191,7 @@ def feature(self, node="clickhouse1"): node.query("REVOKE ADMIN OPTION FOR role0 FROM user0") with Scenario("I revoke admin option for multiple roles from multiple users", flags=TE, requirements=[ - RQ_SRS_006_RBAC_Revoke_Role("1.0"), + RQ_SRS_006_RBAC_Revoke_Role("1.0"), RQ_SRS_006_RBAC_Revoke_AdminOption("1.0")]): with setup(): with When("I revoke admin option for multiple roles from multiple users"): diff --git a/tests/testflows/rbac/tests/syntax/set_default_role.py b/tests/testflows/rbac/tests/syntax/set_default_role.py old mode 100644 new mode 100755 index ec0d41554da..505678a671e --- a/tests/testflows/rbac/tests/syntax/set_default_role.py +++ b/tests/testflows/rbac/tests/syntax/set_default_role.py @@ -34,7 +34,7 @@ def feature(self, node="clickhouse1"): with And("I drop the roles"): for i in range(roles): node.query(f"DROP ROLE IF EXISTS role{i}") - + with Scenario("I set default a nonexistent role to user", requirements=[ RQ_SRS_006_RBAC_SetDefaultRole("1.0")]): with setup(1,0): @@ -56,7 +56,7 @@ def feature(self, node="clickhouse1"): exitcode, message = errors.user_not_found_in_disk(name="user0") node.query("SET DEFAULT ROLE role0 TO user0", exitcode=exitcode, message=message) - #in SET DEFAULT ROLE, the nonexistent user is noticed first and becomes the thrown exception + # In SET DEFAULT ROLE, the nonexistent user is noticed first and becomes the thrown exception with Scenario("I set default a nonexistent role to a nonexistent user", requirements=[ RQ_SRS_006_RBAC_SetDefaultRole("1.0")]): with setup(0,0): diff --git a/tests/testflows/rbac/tests/syntax/set_role.py b/tests/testflows/rbac/tests/syntax/set_role.py old mode 100644 new mode 100755 index fdd1c95112d..813bc5d259c --- a/tests/testflows/rbac/tests/syntax/set_role.py +++ b/tests/testflows/rbac/tests/syntax/set_role.py @@ -22,7 +22,7 @@ def feature(self, node="clickhouse1"): try: with Given("I have some roles"): for i in range(roles): - node.query(f"CREATE ROLE role{i}") + node.query(f"CREATE ROLE role{i}") yield finally: with Finally("I drop the roles"): @@ -54,7 +54,7 @@ def feature(self, node="clickhouse1"): with When("I set nonexistent role for current user"): exitcode, message = errors.role_not_found_in_disk("role5") node.query("SET ROLE ALL EXCEPT role5", exitcode=exitcode, message=message) - + with Scenario("I set one role for current user", flags = TE, requirements=[ RQ_SRS_006_RBAC_SetRole("1.0")]): with setup(1): diff --git a/tests/testflows/rbac/tests/syntax/show_create_quota.py b/tests/testflows/rbac/tests/syntax/show_create_quota.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_create_role.py b/tests/testflows/rbac/tests/syntax/show_create_role.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_create_row_policy.py b/tests/testflows/rbac/tests/syntax/show_create_row_policy.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_create_settings_profile.py b/tests/testflows/rbac/tests/syntax/show_create_settings_profile.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_create_user.py b/tests/testflows/rbac/tests/syntax/show_create_user.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_grants.py b/tests/testflows/rbac/tests/syntax/show_grants.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_quotas.py b/tests/testflows/rbac/tests/syntax/show_quotas.py old mode 100644 new mode 100755 diff --git a/tests/testflows/rbac/tests/syntax/show_row_policies.py b/tests/testflows/rbac/tests/syntax/show_row_policies.py old mode 100644 new mode 100755