diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index e20fd89aafa..1d099580502 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=( @@ -178,9 +186,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/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/programs/server/Server.cpp b/programs/server/Server.cpp index e619511d3f1..2bd70efb16b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -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/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/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/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/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/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/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/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/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/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_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