Merge remote-tracking branch 'upstream/master' into fix25

This commit is contained in:
proller 2019-08-27 14:04:16 +03:00
commit ed551117cb
101 changed files with 520 additions and 331 deletions

View File

@ -82,6 +82,8 @@ namespace
template <typename T>
static bool isZero(const T &, const State & /*state*/)
{
/// Careful: apparently this uses SFINAE to redefine isZero for all types
/// except the IndexType, for which the default ZeroTraits::isZero is used.
static_assert(!std::is_same_v<typename std::decay<T>::type, typename std::decay<IndexType>::type>);
return false;
}
@ -122,19 +124,88 @@ namespace
};
/**
* ReverseIndexHashTableBase implements a special hash table interface for
* reverse index.
*
* The following requirements are different compared to a plain hash table:
*
* 1) Provide public access to 'hash table state' that contains
* additional data needed to calculate cell hashes.
*
* 2) Support emplace() and find() with a Key different from the resulting
* hash table key. This means emplace() accepts a different kind of object
* as a key, and then the real key can be read from the returned cell iterator.
*
* These requirements are unique to ReverseIndex and are in conflict with
* supporting hash tables that use alternative key storage, such as FixedHashMap
* or StringHashMap. Therefore, we implement an interface for ReverseIndex
* separately.
*/
template <typename Key, typename Cell, typename Hash>
class HashTableWithPublicState : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
class ReverseIndexHashTableBase : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
{
using State = typename Cell::State;
using Base = HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>;
public:
using Base::Base;
using iterator = typename Base::iterator;
State & getState() { return *this; }
template <typename ObjectToCompareWith>
size_t ALWAYS_INLINE reverseIndexFindCell(const ObjectToCompareWith & x,
size_t hash_value, size_t place_value) const
{
while (!this->buf[place_value].isZero(*this)
&& !this->buf[place_value].keyEquals(x, hash_value, *this))
{
place_value = this->grower.next(place_value);
}
return place_value;
}
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplaceNonZero(const Key & key, iterator & it,
bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
size_t place_value = reverseIndexFindCell(object, hash_value,
this->grower.place(hash_value));
// emplaceNonZeroImpl() might need to re-find the cell if the table grows,
// but it will find it correctly by the key alone, so we don't have to
// pass it the 'object'.
this->emplaceNonZeroImpl(place_value, key, it, inserted, hash_value);
}
/// Searches position by object.
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplace(Key key, iterator & it, bool & inserted,
size_t hash_value, const ObjectToCompareWith& object)
{
if (!this->emplaceIfZero(key, it, inserted, hash_value))
{
reverseIndexEmplaceNonZero(key, it, inserted, hash_value, object);
}
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE reverseIndexFind(ObjectToCompareWith x, size_t hash_value)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? this->iteratorToZero() : this->end();
size_t place_value = reverseIndexFindCell(x, hash_value,
this->grower.place(hash_value));
return !this->buf[place_value].isZero(*this)
? iterator(this, &this->buf[place_value])
: this->end();
}
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexStringHashTable : public HashTableWithPublicState<
class ReverseIndexStringHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
@ -145,7 +216,7 @@ namespace
has_base_index>,
ReverseIndexHash>
{
using Base = HashTableWithPublicState<
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
@ -167,7 +238,7 @@ namespace
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexNumberHashTable : public HashTableWithPublicState<
class ReverseIndexNumberHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
@ -178,7 +249,7 @@ namespace
has_base_index>,
ReverseIndexHash>
{
using Base = HashTableWithPublicState<
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
@ -357,7 +428,7 @@ void ReverseIndex<IndexType, ColumnType>::buildIndex()
else
hash = getHash(column->getDataAt(row));
index->emplace(row + base_index, iterator, inserted, hash, column->getDataAt(row));
index->reverseIndexEmplace(row + base_index, iterator, inserted, hash, column->getDataAt(row));
if (!inserted)
throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR);
@ -402,7 +473,7 @@ UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
else
column->insertData(data.data, data.size);
index->emplace(num_rows + base_index, iterator, inserted, hash, data);
index->reverseIndexEmplace(num_rows + base_index, iterator, inserted, hash, data);
if constexpr (use_saved_hash)
{
@ -428,7 +499,7 @@ UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef &
IteratorType iterator;
auto hash = getHash(data);
iterator = index->find(data, hash);
iterator = index->reverseIndexFind(data, hash);
return iterator == index->end() ? size() + base_index : iterator->getValue();
}

View File

@ -294,26 +294,22 @@ public:
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted) { emplaceImpl(x, it, inserted); }
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t) { emplaceImpl(x, it, inserted); }
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x)
iterator ALWAYS_INLINE find(Key x)
{
return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end();
}
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const
const_iterator ALWAYS_INLINE find(Key x) const
{
return !buf[x].isZero(*this) ? const_iterator(this, &buf[x]) : end();
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value)
iterator ALWAYS_INLINE find(Key, size_t hash_value)
{
return !buf[hash_value].isZero(*this) ? iterator(this, &buf[hash_value]) : end();
}
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value) const
const_iterator ALWAYS_INLINE find(Key, size_t hash_value) const
{
return !buf[hash_value].isZero(*this) ? const_iterator(this, &buf[hash_value]) : end();
}

View File

@ -280,8 +280,7 @@ protected:
#endif
/// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain.
template <typename ObjectToCompareWith>
size_t ALWAYS_INLINE findCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const
size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const
{
while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this))
{
@ -700,13 +699,6 @@ protected:
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
}
/// Same but find place using object. Hack for ReverseIndex.
template <typename ObjectToCompareWith>
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
size_t place_value = findCell(object, hash_value, grower.place(hash_value));
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
}
public:
@ -763,14 +755,6 @@ public:
emplaceNonZero(x, it, inserted, hash_value);
}
/// Same, but search position by object. Hack for ReverseIndex.
template <typename ObjectToCompareWith>
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
if (!emplaceIfZero(x, it, inserted, hash_value))
emplaceNonZero(x, it, inserted, hash_value, object);
}
/// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet.
void ALWAYS_INLINE insertUniqueNonZero(const Cell * cell, size_t hash_value)
{
@ -783,9 +767,7 @@ public:
resize();
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x)
iterator ALWAYS_INLINE find(Key x)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
@ -796,8 +778,7 @@ public:
}
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const
const_iterator ALWAYS_INLINE find(Key x) const
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
@ -808,8 +789,7 @@ public:
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value)
iterator ALWAYS_INLINE find(Key x, size_t hash_value)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
@ -819,8 +799,7 @@ public:
}
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) const
const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();

View File

@ -775,6 +775,7 @@ using ASCIICaseInsensitiveStringSearcher = StringSearcher<false, true>;
using UTF8CaseSensitiveStringSearcher = StringSearcher<true, false>;
using UTF8CaseInsensitiveStringSearcher = StringSearcher<false, false>;
using ASCIICaseSensitiveTokenSearcher = TokenSearcher<ASCIICaseSensitiveStringSearcher>;
using ASCIICaseInsensitiveTokenSearcher = TokenSearcher<ASCIICaseInsensitiveStringSearcher>;
/** Uses functions from libc.

View File

@ -652,7 +652,8 @@ using VolnitskyUTF8 = VolnitskyBase<true, false, ASCIICaseSensitiveStringSearche
using VolnitskyCaseInsensitive = VolnitskyBase<false, true, ASCIICaseInsensitiveStringSearcher>; /// ignores non-ASCII bytes
using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase<false, false, UTF8CaseInsensitiveStringSearcher>;
using VolnitskyToken = VolnitskyBase<true, true, ASCIICaseSensitiveTokenSearcher>;
using VolnitskyCaseSensitiveToken = VolnitskyBase<true, true, ASCIICaseSensitiveTokenSearcher>;
using VolnitskyCaseInsensitiveToken = VolnitskyBase<false, true, ASCIICaseInsensitiveTokenSearcher>;
using MultiVolnitsky = MultiVolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>;
using MultiVolnitskyUTF8 = MultiVolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>;

View File

@ -80,7 +80,7 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_
{
UInt8 bytes_to_skip = source_size % delta_bytes_size;
dest[0] = delta_bytes_size;
dest[1] = bytes_to_skip;
dest[1] = bytes_to_skip; /// unused (backward compatibility)
memcpy(&dest[2], source, bytes_to_skip);
size_t start_pos = 2 + bytes_to_skip;
switch (delta_bytes_size)
@ -101,10 +101,16 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_
return 1 + 1 + source_size;
}
void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const
void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
{
if (source_size < 2)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
UInt8 bytes_size = source[0];
UInt8 bytes_to_skip = source[1];
UInt8 bytes_to_skip = uncompressed_size % bytes_size;
if (UInt32(2 + bytes_to_skip) > source_size)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
memcpy(dest, &source[2], bytes_to_skip);
UInt32 source_size_no_header = source_size - bytes_to_skip - 2;

View File

@ -304,7 +304,7 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s
{
UInt8 bytes_to_skip = source_size % data_bytes_size;
dest[0] = data_bytes_size;
dest[1] = bytes_to_skip;
dest[1] = bytes_to_skip; /// unused (backward compatibility)
memcpy(&dest[2], source, bytes_to_skip);
size_t start_pos = 2 + bytes_to_skip;
UInt32 compressed_size = 0;
@ -328,10 +328,16 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s
return 1 + 1 + compressed_size;
}
void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const
void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
{
if (source_size < 2)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
UInt8 bytes_size = source[0];
UInt8 bytes_to_skip = source[1];
UInt8 bytes_to_skip = uncompressed_size % bytes_size;
if (UInt32(2 + bytes_to_skip) > source_size)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
memcpy(dest, &source[2], bytes_to_skip);
UInt32 source_size_no_header = source_size - bytes_to_skip - 2;

View File

@ -264,7 +264,7 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc
{
UInt8 bytes_to_skip = source_size % data_bytes_size;
dest[0] = data_bytes_size;
dest[1] = bytes_to_skip;
dest[1] = bytes_to_skip; /// unused (backward compatibility)
memcpy(&dest[2], source, bytes_to_skip);
size_t start_pos = 2 + bytes_to_skip;
UInt32 result_size = 0;
@ -289,10 +289,16 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc
return 1 + 1 + result_size;
}
void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const
void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
{
if (source_size < 2)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
UInt8 bytes_size = source[0];
UInt8 bytes_to_skip = source[1];
UInt8 bytes_to_skip = uncompressed_size % bytes_size;
if (UInt32(2 + bytes_to_skip) > source_size)
throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS);
memcpy(dest, &source[2], bytes_to_skip);
UInt32 source_size_no_header = source_size - bytes_to_skip - 2;

View File

@ -88,6 +88,9 @@ void CompressionCodecMultiple::useInfoAboutType(DataTypePtr data_type)
void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const
{
if (source_size < 1 || !source[0])
throw Exception("Wrong compression methods list", ErrorCodes::CORRUPTED_DATA);
UInt8 compression_methods_size = source[0];
PODArray<char> compressed_buf(&source[compression_methods_size + 1], &source[source_size]);
@ -103,7 +106,8 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour
UInt32 uncompressed_size = ICompressionCodec::readDecompressedBlockSize(compressed_buf.data());
if (idx == 0 && uncompressed_size != decompressed_size)
throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA);
throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) +
", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA);
uncompressed_buf.resize(uncompressed_size + codec->getAdditionalSizeAtTheEndOfBuffer());
codec->decompress(compressed_buf.data(), source_size, uncompressed_buf.data());

View File

@ -47,7 +47,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch
throw Exception("Can't decompress data with codec byte " + toString(method) + " from codec with byte " + toString(method), ErrorCodes::CANNOT_DECOMPRESS);
UInt8 header_size = getHeaderSize();
UInt32 decompressed_size = unalignedLoad<UInt32>(&source[5]);
UInt32 decompressed_size = readDecompressedBlockSize(source);
doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size);
return decompressed_size;

View File

@ -333,7 +333,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \
M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \
\
M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \
M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \
\
M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \
M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \

View File

@ -436,7 +436,7 @@ struct MultiSearchFirstIndexImpl
/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation.
*/
template <bool negate_result = false>
template <typename TokenSearcher, bool negate_result = false>
struct HasTokenImpl
{
using ResultType = UInt8;
@ -454,7 +454,7 @@ struct HasTokenImpl
/// The current index in the array of strings.
size_t i = 0;
VolnitskyToken searcher(pattern.data(), pattern.size(), end - pos);
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
@ -483,7 +483,7 @@ struct HasTokenImpl
static void constant_constant(const std::string & data, const std::string & pattern, UInt8 & res)
{
VolnitskyToken searcher(pattern.data(), pattern.size(), data.size());
TokenSearcher searcher(pattern.data(), pattern.size(), data.size());
const auto found = searcher.search(data.c_str(), data.size()) != data.end().base();
res = negate_result ^ found;
}
@ -589,6 +589,11 @@ struct NameHasToken
static constexpr auto name = "hasToken";
};
struct NameHasTokenCaseInsensitive
{
static constexpr auto name = "hasTokenCaseInsensitive";
};
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
@ -615,7 +620,8 @@ using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch<MultiSea
using FunctionMultiSearchFirstPositionCaseInsensitive = FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<false>, NameHasToken>;
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
using FunctionHasTokenCaseInsensitive = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
void registerFunctionsStringSearch(FunctionFactory & factory)
{
@ -645,6 +651,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory)
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
factory.registerFunction<FunctionHasToken>();
factory.registerFunction<FunctionHasTokenCaseInsensitive>();
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
}

View File

@ -142,7 +142,7 @@ struct ContextShared
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
MergeTreeSettingsPtr merge_tree_settings; /// Settings of MergeTree* engines.
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
@ -1759,9 +1759,9 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const
if (!shared->merge_tree_settings)
{
auto & config = getConfigRef();
MutableMergeTreeSettingsPtr settings_ptr = MergeTreeSettings::create();
settings_ptr->loadFromConfig("merge_tree", config);
shared->merge_tree_settings = std::move(settings_ptr);
MergeTreeSettings mt_settings;
mt_settings.loadFromConfig("merge_tree", config);
shared->merge_tree_settings.emplace(mt_settings);
}
return *shared->merge_tree_settings;

View File

@ -104,7 +104,7 @@ BlockIO InterpreterAlterQuery::execute()
{
auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId());
alter_commands.validate(*table, context);
table->alter(alter_commands, database_name, table_name, context, table_lock_holder);
table->alter(alter_commands, context, table_lock_holder);
}
return {};

View File

@ -30,6 +30,7 @@ namespace DB
/// Simplified version of the StorageDistributed class.
class StorageDistributedFake : public ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
{
friend struct ext::shared_ptr_helper<StorageDistributedFake>;
public:
std::string getName() const override { return "DistributedFake"; }
bool isRemote() const override { return true; }

View File

@ -372,11 +372,12 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
void IStorage::alterSettings(
const SettingsChanges & new_changes,
const String & current_database_name,
const String & current_table_name,
const Context & context,
TableStructureWriteLockHolder & /* table_lock_holder */)
{
const String current_database_name = getDatabaseName();
const String current_table_name = getTableName();
IDatabase::ASTModifier storage_modifier = [&] (IAST & ast)
{
if (!new_changes.empty())
@ -404,16 +405,16 @@ void IStorage::alterSettings(
void IStorage::alter(
const AlterCommands & params,
const String & database_name,
const String & table_name,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder)
{
const String database_name = getDatabaseName();
const String table_name = getTableName();
if (params.isSettingsAlter())
{
SettingsChanges new_changes;
params.applyForSettingsOnly(new_changes);
alterSettings(new_changes, database_name, table_name, context, table_lock_holder);
alterSettings(new_changes, context, table_lock_holder);
return;
}

View File

@ -281,7 +281,7 @@ public:
* This method must fully execute the ALTER query, taking care of the locks itself.
* To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata.
*/
virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder);
virtual void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder);
/** ALTER tables with regard to its partitions.
* Should handle locks for each command on its own.
@ -295,8 +295,6 @@ public:
*/
virtual void alterSettings(
const SettingsChanges & new_changes,
const String & current_database_name,
const String & current_table_name,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder);

View File

@ -414,8 +414,6 @@ bool StorageKafka::hasSetting(const String & setting_name) const
void StorageKafka::alterSettings(
const SettingsChanges & /* new_changes */,
const String & /* current_database_name */,
const String & /* current_table_name */,
const Context & /* context */,
TableStructureWriteLockHolder & /* table_lock_holder */)
{

View File

@ -20,6 +20,7 @@ namespace DB
*/
class StorageKafka : public ext::shared_ptr_helper<StorageKafka>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageKafka>;
public:
std::string getName() const override { return "Kafka"; }
std::string getTableName() const override { return table_name; }
@ -61,8 +62,6 @@ public:
void alterSettings(
const SettingsChanges & new_changes,
const String & current_database_name,
const String & current_table_name,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder) override;

View File

@ -54,7 +54,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
String part_name = params.get("part");
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
/// Validation of the input that may come from malicious replica.
MergeTreePartInfo::fromPartName(part_name, data.format_version);
@ -175,7 +175,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
{
/// Validation of the input that may come from malicious replica.
MergeTreePartInfo::fromPartName(part_name, data.format_version);
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
Poco::URI uri;
uri.setScheme(interserver_scheme);

View File

@ -39,7 +39,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream(
, compute_granularity(index_granularity.empty())
, codec(std::move(codec_))
, skip_indices(indices_to_recalc)
, with_final_mark(storage.getCOWSettings()->write_final_mark && can_use_adaptive_granularity)
, with_final_mark(storage.getSettings()->write_final_mark && can_use_adaptive_granularity)
{
if (blocks_are_granules_size && !index_granularity.empty())
throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR);
@ -139,7 +139,7 @@ void fillIndexGranularityImpl(
void IMergedBlockOutputStream::fillIndexGranularity(const Block & block)
{
const auto storage_settings = storage.getCOWSettings();
const auto storage_settings = storage.getSettings();
fillIndexGranularityImpl(
block,
storage_settings->index_granularity_bytes,

View File

@ -107,7 +107,7 @@ MergeTreeData::MergeTreeData(
const ASTPtr & sample_by_ast_,
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
MergeTreeSettingsPtr settings_,
std::unique_ptr<MergeTreeSettings> storage_settings_,
bool require_part_metadata_,
bool attach,
BrokenPartCallback broken_part_callback_)
@ -121,11 +121,11 @@ MergeTreeData::MergeTreeData(
full_path(full_path_),
broken_part_callback(broken_part_callback_),
log_name(database_name + "." + table_name), log(&Logger::get(log_name)),
guarded_settings(settings_),
storage_settings(std::move(storage_settings_)),
data_parts_by_info(data_parts_indexes.get<TagByInfo>()),
data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_);
/// NOTE: using the same columns list as is read when performing actual merges.
@ -732,7 +732,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
{
LOG_DEBUG(log, "Loading data parts");
const auto settings = getCOWSettings();
const auto settings = getSettings();
Strings part_file_names;
Poco::DirectoryIterator end;
for (Poco::DirectoryIterator it(full_path); it != end; ++it)
@ -965,7 +965,7 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life
if (!lock.try_lock())
return;
const auto settings = getCOWSettings();
const auto settings = getSettings();
time_t current_time = time(nullptr);
ssize_t deadline = (custom_directories_lifetime_seconds >= 0)
? current_time - custom_directories_lifetime_seconds
@ -1020,7 +1020,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts()
if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example).
part_remove_time < now &&
now - part_remove_time > getCOWSettings()->old_parts_lifetime.totalSeconds())
now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds())
{
parts_to_delete.emplace_back(it);
}
@ -1104,7 +1104,7 @@ void MergeTreeData::clearOldPartsFromFilesystem()
void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_remove)
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
if (parts_to_remove.size() > 1 && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold)
{
/// Parallel parts removal.
@ -1341,7 +1341,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
const IndicesASTs & old_indices, const IndicesASTs & new_indices, ExpressionActionsPtr & out_expression,
NameToNameMap & out_rename_map, bool & out_force_update_metadata) const
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
out_expression = nullptr;
out_rename_map = {};
out_force_update_metadata = false;
@ -1507,7 +1507,7 @@ void MergeTreeData::alterDataPart(
bool skip_sanity_checks,
AlterDataPartTransactionPtr & transaction)
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
ExpressionActionsPtr expression;
const auto & part = transaction->getDataPart();
bool force_update_metadata;
@ -1647,16 +1647,16 @@ void MergeTreeData::alterDataPart(
void MergeTreeData::alterSettings(
const SettingsChanges & new_changes,
const String & current_database_name,
const String & current_table_name,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder)
{
std::lock_guard lock(settings_mutex);
MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate();
settings->updateFromChanges(new_changes);
IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder);
guarded_settings.setPtr(std::move(settings));
const String current_database_name = getDatabaseName();
const String current_table_name = getTableName();
MergeTreeSettings copy = *getSettings();
copy.updateFromChanges(new_changes);
IStorage::alterSettings(new_changes, context, table_lock_holder);
storage_settings.set(std::make_unique<const MergeTreeSettings>(copy));
}
bool MergeTreeData::hasSetting(const String & setting_name) const
@ -2342,7 +2342,7 @@ std::optional<Int64> MergeTreeData::getMinPartDataVersion() const
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
const size_t parts_count_in_total = getPartsCount();
if (parts_count_in_total >= settings->max_parts_in_total)
{
@ -2380,7 +2380,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const
void MergeTreeData::throwInsertIfNeeded() const
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
const size_t parts_count_in_total = getPartsCount();
if (parts_count_in_total >= settings->max_parts_in_total)
{
@ -3075,7 +3075,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String &
bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
if (!settings->enable_mixed_granularity_parts || settings->index_granularity_bytes == 0)
{

View File

@ -331,7 +331,7 @@ public:
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
MergeTreeSettingsPtr settings_,
std::unique_ptr<MergeTreeSettings> settings_,
bool require_part_metadata_,
bool attach,
BrokenPartCallback broken_part_callback_ = [](const String &){});
@ -541,8 +541,6 @@ public:
/// Not atomic, have to be done with alter intention lock.
void alterSettings(
const SettingsChanges & new_changes,
const String & current_database_name,
const String & current_table_name,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder) override;
@ -620,7 +618,7 @@ public:
/// Has additional constraint in replicated version
virtual bool canUseAdaptiveGranularity() const
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
return settings->index_granularity_bytes != 0 &&
(settings->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts);
}
@ -683,13 +681,12 @@ public:
bool has_non_adaptive_index_granularity_parts = false;
/// Get copy-on-write pointer to storage settings.
/// Get constant pointer to storage settings.
/// Copy this pointer into your scope and you will
/// get consistent settings.
const MergeTreeSettingsPtr getCOWSettings() const
MergeTreeSettingsPtr getSettings() const
{
std::shared_lock lock(settings_mutex);
return guarded_settings.copyPtr();
return storage_settings.get();
}
protected:
@ -721,26 +718,9 @@ protected:
String log_name;
Logger * log;
/// Just hides settings pointer from direct usage
class MergeTreeSettingsGuard
{
private:
/// Settings COW pointer. Data maybe changed at any point of time.
/// If you need consistent settings, just copy pointer to your scope.
MergeTreeSettingsPtr settings_ptr;
public:
MergeTreeSettingsGuard(MergeTreeSettingsPtr settings_ptr_)
: settings_ptr(settings_ptr_)
{}
const MergeTreeSettingsPtr copyPtr() const { return settings_ptr; }
MergeTreeSettingsPtr getPtr() { return settings_ptr; }
void setPtr(MergeTreeSettingsPtr ptr) { settings_ptr = ptr; }
};
/// Storage settings. Don't use this field directly, if you
/// want readonly settings. Prefer getCOWSettings() method.
MergeTreeSettingsGuard guarded_settings;
/// Storage settings.
/// Use get and set to receive readonly versions.
MultiVersion<MergeTreeSettings> storage_settings;
/// Work with data parts
@ -829,7 +809,6 @@ protected:
/// The same for clearOldTemporaryDirectories.
std::mutex clear_old_temporary_directories_mutex;
/// Mutex for settings usage
mutable std::shared_mutex settings_mutex;
void setProperties(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast,
const ColumnsDescription & new_columns,

View File

@ -141,7 +141,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz
throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR);
size_t free_entries = pool_size - pool_used;
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
UInt64 max_size = 0;
if (free_entries >= data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge)
@ -159,7 +159,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz
UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation()
{
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
size_t total_threads_in_pool = pool.getNumberOfThreads();
size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed);
@ -179,7 +179,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
String * out_disable_reason)
{
MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
if (data_parts.empty())
{
@ -556,7 +556,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
Names all_column_names = data.getColumns().getNamesOfPhysical();
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
NamesAndTypesList gathering_columns, merging_columns;
Names gathering_column_names, merging_column_names;
@ -965,7 +965,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
const auto & updated_header = mutations_interpreter.getUpdatedHeader();
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
Block in_header = in->getHeader();
@ -1145,7 +1145,7 @@ MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMer
const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound,
const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const
{
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
if (deduplicate)
return MergeAlgorithm::Horizontal;

View File

@ -673,7 +673,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
size_t sum_marks = 0;
size_t total_rows = 0;
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
size_t adaptive_parts = 0;
for (size_t i = 0; i < parts.size(); ++i)
{
@ -832,7 +832,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO
SortingInfoPtr sorting_info = query_info.sorting_info;
size_t adaptive_parts = 0;
std::vector<size_t> sum_marks_in_parts(parts.size());
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
for (size_t i = 0; i < parts.size(); ++i)
{
@ -1035,7 +1035,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal
const Names & virt_columns,
const Settings & settings) const
{
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
size_t sum_marks = 0;
size_t adaptive_parts = 0;
for (size_t i = 0; i < parts.size(); ++i)

View File

@ -25,7 +25,7 @@ std::optional<std::string> MergeTreeIndexGranularityInfo::getMrkExtensionFromFS(
MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(
const MergeTreeData & storage)
{
const auto storage_settings = storage.getCOWSettings();
const auto storage_settings = storage.getSettings();
fixed_index_granularity = storage_settings->index_granularity;
/// Granularity is fixed
if (!storage.canUseAdaptiveGranularity())

View File

@ -75,9 +75,4 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def)
#undef ADD_IF_ABSENT
}
MergeTreeSettings::MutablePtr MergeTreeSettings::clone() const
{
return COW::create(*this);
}
}

View File

@ -2,7 +2,6 @@
#include <Core/Defines.h>
#include <Core/SettingsCommon.h>
#include <Common/COW.h>
namespace Poco
@ -22,11 +21,9 @@ class ASTStorage;
/** Settings for the MergeTree family of engines.
* Could be loaded from config or from a CREATE TABLE query (SETTINGS clause).
*/
struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>, public COW<MergeTreeSettings>
struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
{
friend class COW<MergeTreeSettings>;
/// M (mutable) for normal settings, IM (immutable) for not updateable settings.
#define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \
IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \
@ -101,14 +98,8 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>, public
/// NOTE: will rewrite the AST to add immutable settings.
void loadFromQuery(ASTStorage & storage_def);
MutablePtr clone() const;
private:
MergeTreeSettings() = default;
MergeTreeSettings(const MergeTreeSettings & o) = default;
};
using MergeTreeSettingsPtr = MergeTreeSettings::Ptr;
using MutableMergeTreeSettingsPtr = MergeTreeSettings::MutablePtr;
using MergeTreeSettingsPtr = std::shared_ptr<const MergeTreeSettings>;
}

View File

@ -31,7 +31,7 @@ MergeTreeThreadSelectBlockInputStream::MergeTreeThreadSelectBlockInputStream(
/// Maybe it will make sence to add settings `max_block_size_bytes`
if (max_block_size_rows && !storage.canUseAdaptiveGranularity())
{
size_t fixed_index_granularity = storage.getCOWSettings()->index_granularity;
size_t fixed_index_granularity = storage.getSettings()->index_granularity;
min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1)
/ max_block_size_rows * max_block_size_rows / fixed_index_granularity;
}

View File

@ -27,7 +27,7 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic
void ReplicatedMergeTreeCleanupThread::run()
{
auto storage_settings = storage.getCOWSettings();
auto storage_settings = storage.getSettings();
const auto CLEANUP_SLEEP_MS = storage_settings->cleanup_delay_period * 1000
+ std::uniform_int_distribution<UInt64>(0, storage_settings->cleanup_delay_period_random_add * 1000)(rng);
@ -75,7 +75,7 @@ void ReplicatedMergeTreeCleanupThread::iterate()
void ReplicatedMergeTreeCleanupThread::clearOldLogs()
{
auto zookeeper = storage.getZooKeeper();
auto storage_settings = storage.getCOWSettings();
auto storage_settings = storage.getSettings();
Coordination::Stat stat;
if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat))
@ -287,7 +287,7 @@ struct ReplicatedMergeTreeCleanupThread::NodeWithStat
void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
{
auto zookeeper = storage.getZooKeeper();
auto storage_settings = storage.getCOWSettings();
auto storage_settings = storage.getSettings();
std::vector<NodeWithStat> timed_blocks;
getBlocksSortedByTime(*zookeeper, timed_blocks);
@ -404,7 +404,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
void ReplicatedMergeTreeCleanupThread::clearOldMutations()
{
auto storage_settings = storage.getCOWSettings();
auto storage_settings = storage.getSettings();
if (!storage_settings->finished_mutations_to_keep)
return;

View File

@ -964,7 +964,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
* Setting max_bytes_to_merge_at_max_space_in_pool still working for regular merges,
* because the leader replica does not assign merges of greater size (except OPTIMIZE PARTITION and OPTIMIZE FINAL).
*/
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
bool ignore_max_size = (entry.type == LogEntry::MERGE_PARTS) && (max_source_parts_size == data_settings->max_bytes_to_merge_at_max_space_in_pool);
if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size)

View File

@ -44,7 +44,7 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage
, log(&Logger::get(log_name))
, active_node_identifier(generateActiveNodeIdentifier())
{
const auto storage_settings = storage.getCOWSettings();
const auto storage_settings = storage.getSettings();
check_period_ms = storage_settings->zookeeper_session_expiration_check_period.totalSeconds() * 1000;
/// Periodicity of checking lag of replica.
@ -122,7 +122,7 @@ void ReplicatedMergeTreeRestartingThread::run()
}
time_t current_time = time(nullptr);
const auto storage_settings = storage.getCOWSettings();
const auto storage_settings = storage.getSettings();
if (current_time >= prev_time_of_check_delay + static_cast<time_t>(storage_settings->check_delay_period))
{
/// Find out lag of replicas.
@ -171,7 +171,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
activateReplica();
const auto & zookeeper = storage.getZooKeeper();
const auto storage_settings = storage.getCOWSettings();
const auto storage_settings = storage.getSettings();
storage.cloneReplicaIfNeeded(zookeeper);

View File

@ -27,7 +27,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos];
const auto data_settings = data.getCOWSettings();
const auto data_settings = data.getSettings();
sampling_expression = formattedAST(data.sample_by_ast);
index_granularity = data_settings->index_granularity;
merging_params_mode = static_cast<int>(data.merging_params.mode);

View File

@ -14,6 +14,7 @@ namespace DB
/// A Storage that allows reading from a single MergeTree data part.
class StorageFromMergeTreeDataPart : public ext::shared_ptr_helper<StorageFromMergeTreeDataPart>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageFromMergeTreeDataPart>;
public:
String getName() const override { return "FromMergeTreeDataPart"; }
String getTableName() const override { return part->storage.getTableName() + " (part " + part->name + ")"; }

View File

@ -574,7 +574,9 @@ static StoragePtr create(const StorageFactory::Arguments & args)
ASTPtr sample_by_ast;
ASTPtr ttl_table_ast;
IndicesDescription indices_description;
MutableMergeTreeSettingsPtr storage_settings = MergeTreeSettings::create(args.context.getMergeTreeSettings());
ConstraintsDescription constraints_description;
std::unique_ptr<MergeTreeSettings> storage_settings = std::make_unique<MergeTreeSettings>(args.context.getMergeTreeSettings());
if (is_extended_storage_def)
{

View File

@ -695,10 +695,13 @@ void StorageBuffer::flushThread()
}
void StorageBuffer::alter(const AlterCommands & params, const String & database_name_, const String & table_name_, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
const String database_name_ = getDatabaseName();
const String table_name_ = getTableName();
/// So that no blocks of the old structure remain.
optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context);

View File

@ -39,6 +39,7 @@ class Context;
*/
class StorageBuffer : public ext::shared_ptr_helper<StorageBuffer>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageBuffer>;
friend class BufferBlockInputStream;
friend class BufferBlockOutputStream;
@ -91,8 +92,9 @@ public:
/// The structure of the subordinate table is not checked and does not change.
void alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
~StorageBuffer() override;
private:
String table_name;
@ -145,8 +147,6 @@ protected:
Context & context_,
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_, bool allow_materialized_);
~StorageBuffer() override;
};
}

View File

@ -21,6 +21,7 @@ class ExternalDictionaries;
class StorageDictionary : public ext::shared_ptr_helper<StorageDictionary>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageDictionary>;
public:
std::string getName() const override { return "Dictionary"; }
std::string getTableName() const override { return table_name; }

View File

@ -266,11 +266,8 @@ StoragePtr StorageDistributed::createWithOwnCluster(
ClusterPtr owned_cluster_,
const Context & context_)
{
auto res = ext::shared_ptr_helper<StorageDistributed>::create(
String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false);
auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false);
res->owned_cluster = owned_cluster_;
return res;
}
@ -282,11 +279,8 @@ StoragePtr StorageDistributed::createWithOwnCluster(
ClusterPtr & owned_cluster_,
const Context & context_)
{
auto res = ext::shared_ptr_helper<StorageDistributed>::create(
String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false);
auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false);
res->owned_cluster = owned_cluster_;
return res;
}
@ -392,11 +386,13 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & c
void StorageDistributed::alter(
const AlterCommands & params, const String & current_database_name, const String & current_table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder)
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
const String current_database_name = getDatabaseName();
const String current_table_name = getTableName();
auto new_columns = getColumns();
auto new_indices = getIndices();
auto new_constraints = getConstraints();

View File

@ -29,6 +29,7 @@ class StorageDistributedDirectoryMonitor;
*/
class StorageDistributed : public ext::shared_ptr_helper<StorageDistributed>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageDistributed>;
friend class DistributedBlockOutputStream;
friend class StorageDistributedDirectoryMonitor;
@ -85,8 +86,7 @@ public:
/// in the sub-tables, you need to manually add and delete columns
/// the structure of the sub-table is not checked
void alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
void startup() override;
void shutdown() override;

View File

@ -20,6 +20,7 @@ class StorageFileBlockOutputStream;
class StorageFile : public ext::shared_ptr_helper<StorageFile>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageFile>;
public:
std::string getName() const override { return "File"; }
std::string getTableName() const override { return table_name; }

View File

@ -15,6 +15,7 @@ namespace DB
*/
class StorageHDFS : public ext::shared_ptr_helper<StorageHDFS>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageHDFS>;
public:
String getName() const override { return "HDFS"; }
String getTableName() const override { return table_name; }

View File

@ -22,6 +22,7 @@ using JoinPtr = std::shared_ptr<Join>;
*/
class StorageJoin : public ext::shared_ptr_helper<StorageJoin>, public StorageSetOrJoinBase
{
friend struct ext::shared_ptr_helper<StorageJoin>;
public:
String getName() const override { return "Join"; }

View File

@ -21,6 +21,7 @@ class StorageLog : public ext::shared_ptr_helper<StorageLog>, public IStorage
{
friend class LogBlockInputStream;
friend class LogBlockOutputStream;
friend struct ext::shared_ptr_helper<StorageLog>;
public:
std::string getName() const override { return "Log"; }

View File

@ -11,6 +11,7 @@ namespace DB
class StorageMaterializedView : public ext::shared_ptr_helper<StorageMaterializedView>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageMaterializedView>;
public:
std::string getName() const override { return "MaterializedView"; }
std::string getTableName() const override { return table_name; }

View File

@ -21,6 +21,7 @@ class StorageMemory : public ext::shared_ptr_helper<StorageMemory>, public IStor
{
friend class MemoryBlockInputStream;
friend class MemoryBlockOutputStream;
friend struct ext::shared_ptr_helper<StorageMemory>;
public:
String getName() const override { return "Memory"; }

View File

@ -397,8 +397,7 @@ DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) c
void StorageMerge::alter(
const AlterCommands & params, const String & database_name_, const String & table_name_,
const Context & context, TableStructureWriteLockHolder & table_lock_holder)
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
@ -406,7 +405,7 @@ void StorageMerge::alter(
auto new_indices = getIndices();
auto new_constraints = getConstraints();
params.applyForColumnsOnly(new_columns);
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {});
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {});
setColumns(new_columns);
}

View File

@ -14,6 +14,7 @@ namespace DB
*/
class StorageMerge : public ext::shared_ptr_helper<StorageMerge>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageMerge>;
public:
std::string getName() const override { return "Merge"; }
std::string getTableName() const override { return table_name; }
@ -47,8 +48,7 @@ public:
/// you need to add and remove columns in the sub-tables manually
/// the structure of sub-tables is not checked
void alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override;

View File

@ -63,14 +63,14 @@ StorageMergeTree::StorageMergeTree(
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
MergeTreeSettingsPtr settings_,
std::unique_ptr<MergeTreeSettings> storage_settings_,
bool has_force_restore_data_flag)
: MergeTreeData(database_name_, table_name_,
path_ + escapeForFileName(table_name_) + '/',
columns_, indices_, constraints_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, ttl_table_ast_, merging_params_,
settings_, false, attach),
std::move(storage_settings_), false, attach),
path(path_),
background_pool(context_.getBackgroundPool()),
reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool())
@ -245,11 +245,12 @@ std::vector<MergeTreeData::AlterDataPartTransactionPtr> StorageMergeTree::prepar
void StorageMergeTree::alter(
const AlterCommands & params,
const String & current_database_name,
const String & current_table_name,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder)
{
const String current_database_name = getDatabaseName();
const String current_table_name = getTableName();
if (!params.isMutable())
{
SettingsChanges new_changes;
@ -257,7 +258,7 @@ void StorageMergeTree::alter(
if (params.isSettingsAlter())
{
params.applyForSettingsOnly(new_changes);
alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder);
alterSettings(new_changes, context, table_lock_holder);
return;
}
@ -804,7 +805,7 @@ Int64 StorageMergeTree::getCurrentMutationVersion(
void StorageMergeTree::clearOldMutations()
{
const auto settings = getCOWSettings();
const auto settings = getSettings();
if (!settings->finished_mutations_to_keep)
return;

View File

@ -23,6 +23,7 @@ namespace DB
*/
class StorageMergeTree : public ext::shared_ptr_helper<StorageMergeTree>, public MergeTreeData
{
friend struct ext::shared_ptr_helper<StorageMergeTree>;
public:
void startup() override;
void shutdown() override;
@ -59,9 +60,7 @@ public:
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
void alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
void checkTableCanBeDropped() const override;
@ -153,7 +152,7 @@ protected:
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
MergeTreeSettingsPtr settings_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag);
};

View File

@ -18,6 +18,7 @@ namespace DB
*/
class StorageMySQL : public ext::shared_ptr_helper<StorageMySQL>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageMySQL>;
public:
StorageMySQL(
const std::string & database_name_,

View File

@ -31,11 +31,13 @@ void registerStorageNull(StorageFactory & factory)
}
void StorageNull::alter(
const AlterCommands & params, const String & current_database_name, const String & current_table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder)
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
const String current_database_name = getDatabaseName();
const String current_table_name = getTableName();
ColumnsDescription new_columns = getColumns();
IndicesDescription new_indices = getIndices();
ConstraintsDescription new_constraints = getConstraints();

View File

@ -16,6 +16,7 @@ namespace DB
*/
class StorageNull : public ext::shared_ptr_helper<StorageNull>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageNull>;
public:
std::string getName() const override { return "Null"; }
std::string getTableName() const override { return table_name; }
@ -44,8 +45,7 @@ public:
}
void alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
private:
String table_name;

View File

@ -202,14 +202,14 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const ASTPtr & sample_by_ast_,
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
MergeTreeSettingsPtr settings_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag)
: MergeTreeData(database_name_, table_name_,
path_ + escapeForFileName(table_name_) + '/',
columns_, indices_, constraints_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, ttl_table_ast_, merging_params_,
settings_, true, attach,
std::move(settings_), true, attach,
[this] (const std::string & name) { enqueuePartForCheck(name); }),
zookeeper_path(global_context.getMacros()->expand(zookeeper_path_, database_name_, table_name_)),
replica_name(global_context.getMacros()->expand(replica_name_, database_name_, table_name_)),
@ -376,7 +376,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
}
/** Verify that list of columns and table storage_settings match those specified in ZK (/ metadata).
/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/ metadata).
* If not, throw an exception.
*/
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
@ -637,8 +637,8 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
for (const auto & part : parts)
total_rows_on_filesystem += part->rows_count;
const auto storage_settings = getCOWSettings();
bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings->replicated_max_ratio_of_wrong_parts;
const auto storage_settings_ptr = getSettings();
bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings_ptr->replicated_max_ratio_of_wrong_parts;
if (insane && !skip_sanity_checks)
{
@ -781,13 +781,13 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
if (!has_been_already_added)
{
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
String part_path = replica_path + "/parts/" + part_name;
ops.emplace_back(zkutil::makeCheckRequest(
zookeeper_path + "/columns", expected_columns_version));
if (storage_settings->use_minimalistic_part_header_in_zookeeper)
if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper)
{
ops.emplace_back(zkutil::makeCreateRequest(
part_path, local_part_header.toString(), zkutil::CreateMode::Persistent));
@ -864,7 +864,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const
{
return MinimalisticDataPartChecksums::getSerializedString(checksums,
getCOWSettings()->use_minimalistic_checksums_in_zookeeper);
getSettings()->use_minimalistic_checksums_in_zookeeper);
}
@ -1035,14 +1035,14 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
parts.push_back(part);
}
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
if (!have_all_parts)
{
/// If you do not have all the necessary parts, try to take some already merged part from someone.
LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead");
return false;
}
else if (entry.create_time + storage_settings->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr))
else if (entry.create_time + storage_settings_ptr->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr))
{
/// If entry is old enough, and have enough size, and part are exists in any replica,
/// then prefer fetching of merged part from replica.
@ -1051,7 +1051,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
for (const auto & part : parts)
sum_parts_bytes_on_disk += part->bytes_on_disk;
if (sum_parts_bytes_on_disk >= storage_settings->prefer_fetch_merged_part_size_threshold)
if (sum_parts_bytes_on_disk >= storage_settings_ptr->prefer_fetch_merged_part_size_threshold)
{
String replica = findReplicaHavingPart(entry.new_part_name, true); /// NOTE excessive ZK requests for same data later, may remove.
if (!replica.empty())
@ -1161,7 +1161,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedMergeTree::LogEntry & entry)
{
const String & source_part_name = entry.source_parts.at(0);
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
LOG_TRACE(log, "Executing log entry to mutate part " << source_part_name << " to " << entry.new_part_name);
DataPartPtr source_part = getActiveContainingPart(source_part_name);
@ -1181,8 +1181,8 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
/// TODO - some better heuristic?
size_t estimated_space_for_result = MergeTreeDataMergerMutator::estimateNeededDiskSpace({source_part});
if (entry.create_time + storage_settings->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)
&& estimated_space_for_result >= storage_settings->prefer_fetch_merged_part_size_threshold)
if (entry.create_time + storage_settings_ptr->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)
&& estimated_space_for_result >= storage_settings_ptr->prefer_fetch_merged_part_size_threshold)
{
/// If entry is old enough, and have enough size, and some replica has the desired part,
/// then prefer fetching from replica.
@ -1276,21 +1276,21 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
{
String replica = findReplicaHavingCoveringPart(entry, true);
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
static std::atomic_uint total_fetches {0};
if (storage_settings->replicated_max_parallel_fetches && total_fetches >= storage_settings->replicated_max_parallel_fetches)
if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches)
{
throw Exception("Too many total fetches from replicas, maximum: " + storage_settings->replicated_max_parallel_fetches.toString(),
throw Exception("Too many total fetches from replicas, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches.toString(),
ErrorCodes::TOO_MANY_FETCHES);
}
++total_fetches;
SCOPE_EXIT({--total_fetches;});
if (storage_settings->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings->replicated_max_parallel_fetches_for_table)
if (storage_settings_ptr->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings_ptr->replicated_max_parallel_fetches_for_table)
{
throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings->replicated_max_parallel_fetches_for_table.toString(),
throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString(),
ErrorCodes::TOO_MANY_FETCHES);
}
@ -2213,7 +2213,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
if (!is_leader)
return;
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
const bool deduplicate = false; /// TODO: read deduplicate option from table config
const bool force_ttl = false;
@ -2235,16 +2235,16 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
auto merges_and_mutations_queued = queue.countMergesAndPartMutations();
size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second;
if (merges_and_mutations_sum >= storage_settings->max_replicated_merges_in_queue)
if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue)
{
LOG_TRACE(log, "Number of queued merges (" << merges_and_mutations_queued.first << ") and part mutations ("
<< merges_and_mutations_queued.second << ") is greater than max_replicated_merges_in_queue ("
<< storage_settings->max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate.");
<< storage_settings_ptr->max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate.");
}
else
{
UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge(
storage_settings->max_replicated_merges_in_queue, merges_and_mutations_sum);
storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum);
UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation();
FutureMergedMutatedPart future_merged_part;
@ -2256,7 +2256,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
}
/// If there are many mutations in queue it may happen, that we cannot enqueue enough merges to merge all new parts
else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0
&& merges_and_mutations_queued.second < storage_settings->max_replicated_mutations_in_queue)
&& merges_and_mutations_queued.second < storage_settings_ptr->max_replicated_mutations_in_queue)
{
/// Choose a part to mutate.
DataPartsVector data_parts = getDataPartsVector();
@ -3029,11 +3029,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const
BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Context & context)
{
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
assertNotReadonly();
const Settings & query_settings = context.getSettingsRef();
bool deduplicate = storage_settings->replicated_deduplication_window != 0 && query_settings.insert_deduplicate;
bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate;
return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(*this,
query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate);
@ -3067,7 +3067,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
};
bool force_ttl = (final && (hasTableTTL() || hasAnyColumnTTL()));
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
if (!partition && final)
{
@ -3100,7 +3100,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
if (!partition)
{
selected = merger_mutator.selectPartsToMerge(
future_merged_part, true, storage_settings->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
}
else
{
@ -3142,21 +3142,23 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
void StorageReplicatedMergeTree::alter(
const AlterCommands & params, const String & current_database_name, const String & current_table_name,
const Context & query_context, TableStructureWriteLockHolder & table_lock_holder)
const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder)
{
assertNotReadonly();
LOG_DEBUG(log, "Doing ALTER");
const String current_database_name = getDatabaseName();
const String current_table_name = getTableName();
if (params.isSettingsAlter())
{
/// We don't replicate storage_settings ALTER. It's local operation.
/// We don't replicate storage_settings_ptr ALTER. It's local operation.
/// Also we don't upgrade alter lock to table structure lock.
LOG_DEBUG(log, "ALTER storage_settings only");
LOG_DEBUG(log, "ALTER storage_settings_ptr only");
SettingsChanges new_changes;
params.applyForSettingsOnly(new_changes);
alterSettings(new_changes, current_database_name, current_table_name, query_context, table_lock_holder);
alterSettings(new_changes, query_context, table_lock_holder);
return;
}
@ -3948,10 +3950,10 @@ void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String &
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
{
auto zookeeper = tryGetZooKeeper();
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
res.is_leader = is_leader;
res.can_become_leader = storage_settings->replicated_can_become_leader;
res.can_become_leader = storage_settings_ptr->replicated_can_become_leader;
res.is_readonly = is_readonly;
res.is_session_expired = !zookeeper || zookeeper->expired();
@ -4141,14 +4143,14 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t
out_absolute_delay = getAbsoluteDelay();
out_relative_delay = 0;
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
/** Relative delay is the maximum difference of absolute delay from any other replica,
* (if this replica lags behind any other live replica, or zero, otherwise).
* Calculated only if the absolute delay is large enough.
*/
if (out_absolute_delay < static_cast<time_t>(storage_settings->min_relative_delay_to_yield_leadership))
if (out_absolute_delay < static_cast<time_t>(storage_settings_ptr->min_relative_delay_to_yield_leadership))
return;
auto zookeeper = getZooKeeper();
@ -4969,7 +4971,7 @@ void StorageReplicatedMergeTree::getCommitPartOps(
const String & block_id_path) const
{
const String & part_name = part->name;
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
if (!block_id_path.empty())
{
@ -4987,7 +4989,7 @@ void StorageReplicatedMergeTree::getCommitPartOps(
zookeeper_path + "/columns",
columns_version));
if (storage_settings->use_minimalistic_part_header_in_zookeeper)
if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper)
{
ops.emplace_back(zkutil::makeCreateRequest(
replica_path + "/parts/" + part->name,
@ -5016,12 +5018,12 @@ void StorageReplicatedMergeTree::updatePartHeaderInZooKeeperAndCommit(
AlterDataPartTransaction & transaction)
{
String part_path = replica_path + "/parts/" + transaction.getPartName();
const auto storage_settings = getCOWSettings();
const auto storage_settings_ptr = getSettings();
bool need_delete_columns_and_checksums_nodes = false;
try
{
if (storage_settings->use_minimalistic_part_header_in_zookeeper)
if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper)
{
auto part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums(
transaction.getNewColumns(), transaction.getNewChecksums());
@ -5201,9 +5203,9 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, const C
bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const
{
const auto storage_settings = getCOWSettings();
return storage_settings->index_granularity_bytes != 0 &&
(storage_settings->enable_mixed_granularity_parts ||
const auto storage_settings_ptr = getSettings();
return storage_settings_ptr->index_granularity_bytes != 0 &&
(storage_settings_ptr->enable_mixed_granularity_parts ||
(!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity));
}

View File

@ -74,6 +74,7 @@ namespace DB
class StorageReplicatedMergeTree : public ext::shared_ptr_helper<StorageReplicatedMergeTree>, public MergeTreeData
{
friend struct ext::shared_ptr_helper<StorageReplicatedMergeTree>;
public:
void startup() override;
void shutdown() override;
@ -98,9 +99,7 @@ public:
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override;
void alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override;
void alter(const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override;
void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override;
@ -540,7 +539,7 @@ protected:
const ASTPtr & sample_by_ast_,
const ASTPtr & table_ttl_ast_,
const MergingParams & merging_params_,
MergeTreeSettingsPtr settings_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag);
};

View File

@ -23,6 +23,7 @@ class StorageStripeLog : public ext::shared_ptr_helper<StorageStripeLog>, public
{
friend class StripeLogBlockInputStream;
friend class StripeLogBlockOutputStream;
friend struct ext::shared_ptr_helper<StorageStripeLog>;
public:
std::string getName() const override { return "StripeLog"; }

View File

@ -22,6 +22,7 @@ class StorageTinyLog : public ext::shared_ptr_helper<StorageTinyLog>, public ISt
{
friend class TinyLogBlockInputStream;
friend class TinyLogBlockOutputStream;
friend struct ext::shared_ptr_helper<StorageTinyLog>;
public:
std::string getName() const override { return "TinyLog"; }

View File

@ -71,6 +71,7 @@ private:
class StorageURL : public ext::shared_ptr_helper<StorageURL>, public IStorageURLBase
{
friend struct ext::shared_ptr_helper<StorageURL>;
public:
StorageURL(
const Poco::URI & uri_,

View File

@ -11,6 +11,7 @@ namespace DB
*/
class StorageValues : public ext::shared_ptr_helper<StorageValues>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageValues>;
public:
std::string getName() const override { return "Values"; }
std::string getTableName() const override { return table_name; }

View File

@ -12,6 +12,7 @@ namespace DB
class StorageView : public ext::shared_ptr_helper<StorageView>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageView>;
public:
std::string getName() const override { return "View"; }
std::string getTableName() const override { return table_name; }

View File

@ -4,11 +4,13 @@
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>,
public IStorageSystemOneBlock<StorageSystemAggregateFunctionCombinators>
{
friend struct ext::shared_ptr_helper<StorageSystemAggregateFunctionCombinators>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -12,8 +12,10 @@ class Context;
/** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics.
*/
class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>, public IStorageSystemOneBlock<StorageSystemAsynchronousMetrics>
class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>,
public IStorageSystemOneBlock<StorageSystemAsynchronousMetrics>
{
friend struct ext::shared_ptr_helper<StorageSystemAsynchronousMetrics>;
public:
std::string getName() const override { return "SystemAsynchronousMetrics"; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemBuildOptions : public ext::shared_ptr_helper<StorageSystemBuildOptions>, public IStorageSystemOneBlock<StorageSystemBuildOptions>
{
friend struct ext::shared_ptr_helper<StorageSystemBuildOptions>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -17,6 +17,7 @@ class Context;
*/
class StorageSystemClusters : public ext::shared_ptr_helper<StorageSystemClusters>, public IStorageSystemOneBlock<StorageSystemClusters>
{
friend struct ext::shared_ptr_helper<StorageSystemClusters>;
public:
std::string getName() const override { return "SystemClusters"; }

View File

@ -8,6 +8,7 @@ namespace DB
class StorageSystemCollations : public ext::shared_ptr_helper<StorageSystemCollations>,
public IStorageSystemOneBlock<StorageSystemCollations>
{
friend struct ext::shared_ptr_helper<StorageSystemCollations>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -13,6 +13,7 @@ class Context;
*/
class StorageSystemColumns : public ext::shared_ptr_helper<StorageSystemColumns>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemColumns>;
public:
std::string getName() const override { return "SystemColumns"; }
std::string getTableName() const override { return name; }

View File

@ -14,6 +14,7 @@ class Context;
class StorageSystemContributors : public ext::shared_ptr_helper<StorageSystemContributors>,
public IStorageSystemOneBlock<StorageSystemContributors>
{
friend struct ext::shared_ptr_helper<StorageSystemContributors>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -9,6 +9,7 @@ namespace DB
class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper<StorageSystemDataTypeFamilies>,
public IStorageSystemOneBlock<StorageSystemDataTypeFamilies>
{
friend struct ext::shared_ptr_helper<StorageSystemDataTypeFamilies>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemDatabases : public ext::shared_ptr_helper<StorageSystemDatabases>, public IStorageSystemOneBlock<StorageSystemDatabases>
{
friend struct ext::shared_ptr_helper<StorageSystemDatabases>;
public:
std::string getName() const override
{

View File

@ -21,6 +21,7 @@ class StorageSystemDetachedParts :
public ext::shared_ptr_helper<StorageSystemDetachedParts>,
public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemDetachedParts>;
public:
std::string getName() const override { return "SystemDetachedParts"; }
std::string getTableName() const override { return "detached_parts"; }

View File

@ -12,6 +12,7 @@ class Context;
class StorageSystemDictionaries : public ext::shared_ptr_helper<StorageSystemDictionaries>, public IStorageSystemOneBlock<StorageSystemDictionaries>
{
friend struct ext::shared_ptr_helper<StorageSystemDictionaries>;
public:
std::string getName() const override { return "SystemDictionaries"; }

View File

@ -13,6 +13,7 @@ class Context;
*/
class StorageSystemEvents : public ext::shared_ptr_helper<StorageSystemEvents>, public IStorageSystemOneBlock<StorageSystemEvents>
{
friend struct ext::shared_ptr_helper<StorageSystemEvents>;
public:
std::string getName() const override { return "SystemEvents"; }

View File

@ -7,6 +7,7 @@ namespace DB
{
class StorageSystemFormats : public ext::shared_ptr_helper<StorageSystemFormats>, public IStorageSystemOneBlock<StorageSystemFormats>
{
friend struct ext::shared_ptr_helper<StorageSystemFormats>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -15,6 +15,7 @@ class Context;
*/
class StorageSystemFunctions : public ext::shared_ptr_helper<StorageSystemFunctions>, public IStorageSystemOneBlock<StorageSystemFunctions>
{
friend struct ext::shared_ptr_helper<StorageSystemFunctions>;
public:
std::string getName() const override { return "SystemFunctions"; }

View File

@ -13,6 +13,7 @@ namespace DB
/// Provides information about Graphite configuration.
class StorageSystemGraphite : public ext::shared_ptr_helper<StorageSystemGraphite>, public IStorageSystemOneBlock<StorageSystemGraphite>
{
friend struct ext::shared_ptr_helper<StorageSystemGraphite>;
public:
std::string getName() const override { return "SystemGraphite"; }

View File

@ -15,6 +15,7 @@ class Context;
*/
class StorageSystemMacros : public ext::shared_ptr_helper<StorageSystemMacros>, public IStorageSystemOneBlock<StorageSystemMacros>
{
friend struct ext::shared_ptr_helper<StorageSystemMacros>;
public:
std::string getName() const override { return "SystemMacros"; }

View File

@ -15,6 +15,7 @@ class Context;
*/
class SystemMergeTreeSettings : public ext::shared_ptr_helper<SystemMergeTreeSettings>, public IStorageSystemOneBlock<SystemMergeTreeSettings>
{
friend struct ext::shared_ptr_helper<SystemMergeTreeSettings>;
public:
std::string getName() const override { return "SystemMergeTreeSettings"; }

View File

@ -15,6 +15,7 @@ class Context;
class StorageSystemMerges : public ext::shared_ptr_helper<StorageSystemMerges>, public IStorageSystemOneBlock<StorageSystemMerges>
{
friend struct ext::shared_ptr_helper<StorageSystemMerges>;
public:
std::string getName() const override { return "SystemMerges"; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemMetrics : public ext::shared_ptr_helper<StorageSystemMetrics>, public IStorageSystemOneBlock<StorageSystemMetrics>
{
friend struct ext::shared_ptr_helper<StorageSystemMetrics>;
public:
std::string getName() const override { return "SystemMetrics"; }

View File

@ -12,6 +12,7 @@ class Context;
class StorageSystemModels : public ext::shared_ptr_helper<StorageSystemModels>, public IStorageSystemOneBlock<StorageSystemModels>
{
friend struct ext::shared_ptr_helper<StorageSystemModels>;
public:
std::string getName() const override { return "SystemModels"; }

View File

@ -14,6 +14,7 @@ class Context;
/// in the MergeTree tables.
class StorageSystemMutations : public ext::shared_ptr_helper<StorageSystemMutations>, public IStorageSystemOneBlock<StorageSystemMutations>
{
friend struct ext::shared_ptr_helper<StorageSystemMutations>;
public:
String getName() const override { return "SystemMutations"; }

View File

@ -25,6 +25,7 @@ class Context;
*/
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemNumbers>;
public:
std::string getName() const override { return "SystemNumbers"; }
std::string getTableName() const override { return name; }

View File

@ -17,6 +17,7 @@ class Context;
*/
class StorageSystemOne : public ext::shared_ptr_helper<StorageSystemOne>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemOne>;
public:
std::string getName() const override { return "SystemOne"; }
std::string getTableName() const override { return name; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemParts : public ext::shared_ptr_helper<StorageSystemParts>, public StorageSystemPartsBase
{
friend struct ext::shared_ptr_helper<StorageSystemParts>;
public:
std::string getName() const override { return "SystemParts"; }

View File

@ -16,6 +16,7 @@ class Context;
class StorageSystemPartsColumns
: public ext::shared_ptr_helper<StorageSystemPartsColumns>, public StorageSystemPartsBase
{
friend struct ext::shared_ptr_helper<StorageSystemPartsColumns>;
public:
std::string getName() const override { return "SystemPartsColumns"; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemProcesses : public ext::shared_ptr_helper<StorageSystemProcesses>, public IStorageSystemOneBlock<StorageSystemProcesses>
{
friend struct ext::shared_ptr_helper<StorageSystemProcesses>;
public:
std::string getName() const override { return "SystemProcesses"; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemReplicas : public ext::shared_ptr_helper<StorageSystemReplicas>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemReplicas>;
public:
std::string getName() const override { return "SystemReplicas"; }
std::string getTableName() const override { return name; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemReplicationQueue : public ext::shared_ptr_helper<StorageSystemReplicationQueue>, public IStorageSystemOneBlock<StorageSystemReplicationQueue>
{
friend struct ext::shared_ptr_helper<StorageSystemReplicationQueue>;
public:
std::string getName() const override { return "SystemReplicationQueue"; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemSettings : public ext::shared_ptr_helper<StorageSystemSettings>, public IStorageSystemOneBlock<StorageSystemSettings>
{
friend struct ext::shared_ptr_helper<StorageSystemSettings>;
public:
std::string getName() const override { return "SystemSettings"; }

View File

@ -10,6 +10,7 @@ namespace DB
class StorageSystemTableEngines : public ext::shared_ptr_helper<StorageSystemTableEngines>,
public IStorageSystemOneBlock<StorageSystemTableEngines>
{
friend struct ext::shared_ptr_helper<StorageSystemTableEngines>;
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;

View File

@ -9,6 +9,7 @@ namespace DB
class StorageSystemTableFunctions : public ext::shared_ptr_helper<StorageSystemTableFunctions>,
public IStorageSystemOneBlock<StorageSystemTableFunctions>
{
friend struct ext::shared_ptr_helper<StorageSystemTableFunctions>;
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemTables : public ext::shared_ptr_helper<StorageSystemTables>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageSystemTables>;
public:
std::string getName() const override { return "SystemTables"; }
std::string getTableName() const override { return name; }

View File

@ -14,6 +14,7 @@ class Context;
*/
class StorageSystemZooKeeper : public ext::shared_ptr_helper<StorageSystemZooKeeper>, public IStorageSystemOneBlock<StorageSystemZooKeeper>
{
friend struct ext::shared_ptr_helper<StorageSystemZooKeeper>;
public:
std::string getName() const override { return "SystemZooKeeper"; }

View File

@ -101,7 +101,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE test.src;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2);" -- trash part to be
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted"
# Stop replication at the second replica and remove source table to use fetch instead of copying
$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;"

View File

@ -2,11 +2,12 @@
# encoding: utf-8
import re
from string import Template
HAYSTACKS = [
"hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay",
"hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay needle",
"needle hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay",
"hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay",
]
NEEDLE = "needle"
@ -48,47 +49,77 @@ def transform_needle(query, string_transformation_func):
return NEEDLE_RE.sub(replace_with_transformation, query)
def create_cases(table_row_template, table_query_template, const_query_template):
def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, table_query_template, const_query_template):
const_queries = []
table_rows = []
table_queries = set()
def add_case(haystack, needle, match):
def add_case(func, haystack, needle, match):
match = int(match)
const_queries.append(const_query_template.format(haystack=haystack, needle=needle, match=match))
table_queries.add(table_query_template.format(haystack=haystack, needle=needle, match=match))
table_rows.append(table_row_template.format(haystack=haystack, needle=needle, match=match))
args = dict(
func = func,
haystack = haystack,
needle = needle,
match = match
)
const_queries.append(const_query_template.substitute(args))
table_queries.add(table_query_template.substitute(args))
table_rows.append(table_row_template.substitute(args))
def add_case_sensitive(haystack, needle, match):
add_case(case_sensitive_func, haystack, needle, match)
if match:
add_case(case_sensitive_func, transform_needle(haystack, str.swapcase), transform_needle(needle, str.swapcase), match)
def add_case_insensitive(haystack, needle, match):
add_case(case_insensitive_func, haystack, needle, match)
if match:
add_case(case_insensitive_func, transform_needle(haystack, str.swapcase), needle, match)
add_case(case_insensitive_func, haystack, transform_needle(needle, str.swapcase), match)
# Negative cases
add_case(remove_needle(HAYSTACKS[0]), NEEDLE, False)
add_case_sensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False)
add_case_insensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False)
for haystack in HAYSTACKS:
add_case(transform_needle(haystack, str.title), NEEDLE, False)
add_case_sensitive(transform_needle(haystack, str.swapcase), NEEDLE, False)
sep = ''
h = replace_separators(haystack, sep)
add_case(h, NEEDLE, False)
add_case(small_needle(h), small_needle(NEEDLE), False)
add_case(enlarge_haystack(h, 10, sep), NEEDLE, False)
add_case_sensitive(h, NEEDLE, False)
add_case_insensitive(h, NEEDLE, False)
add_case_sensitive(small_needle(h), small_needle(NEEDLE), False)
add_case_insensitive(small_needle(h), small_needle(NEEDLE), False)
add_case_sensitive(enlarge_haystack(h, 10, sep), NEEDLE, False)
add_case_insensitive(enlarge_haystack(h, 10, sep), NEEDLE, False)
# positive cases
for haystack in HAYSTACKS:
add_case(transform_needle(haystack, str.title), transform_needle(NEEDLE, str.title), True)
add_case(transform_needle(haystack, str.upper), transform_needle(NEEDLE, str.upper), True)
add_case_sensitive(haystack, NEEDLE, True)
add_case_insensitive(haystack, NEEDLE, True)
# Not checking all separators since some (like ' and \n) cause issues when coupled with
# re-based replacement and quoting in query
# other are rare in practice and checking all separators makes this test too lengthy.
# r'\\\\' turns into a single '\' in query
#separators = list(''' \t`~!@#$%^&*()-=+|]}[{";:/?.>,<''') + [r'\\\\']
separators = list(''' \t;:?.,''') + [r'\\\\']
for sep in separators:
for sep in list(''' ,'''):
h = replace_separators(haystack, sep)
add_case(h, NEEDLE, True)
add_case(small_needle(h), small_needle(NEEDLE), True)
add_case(enlarge_haystack(h, 200, sep), NEEDLE, True)
add_case(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True)
add_case(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True)
add_case_sensitive(h, NEEDLE, True)
add_case_sensitive(small_needle(h), small_needle(NEEDLE), True)
add_case_sensitive(enlarge_haystack(h, 200, sep), NEEDLE, True)
add_case_insensitive(h, NEEDLE, True)
add_case_insensitive(small_needle(h), small_needle(NEEDLE), True)
add_case_insensitive(enlarge_haystack(h, 200, sep), NEEDLE, True)
# case insesitivity works only on ASCII strings
add_case_sensitive(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True)
add_case_sensitive(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True)
for sep in list('''~!@$%^&*()-=+|]}[{";:/?.><\t''') + [r'\\\\']:
h = replace_separators(HAYSTACKS[0], sep)
add_case(case_sensitive_func, h, NEEDLE, True)
return table_rows, table_queries, const_queries
@ -97,12 +128,14 @@ def main():
def query(x):
print x
CONST_QUERY = """SELECT hasToken('{haystack}', '{needle}'), ' expecting ', {match};"""
#SELECT hasToken(haystack, '{needle}') FROM ht WHERE needle = '{needle}' AND match = {match};"""
TABLE_QUERY = """WITH '{needle}' as n SELECT haystack, needle, hasToken(haystack, n) as result FROM ht WHERE needle = n AND result != match;"""
TABLE_ROW = """('{haystack}', '{needle}', {match})"""
CONST_QUERY = Template("""SELECT ${func}('${haystack}', '${needle}'), ' expecting ', ${match};""")
TABLE_QUERY = Template("""WITH '${needle}' as n
SELECT haystack, needle, ${func}(haystack, n) as result
FROM ht
WHERE func = '${func}' AND needle = n AND result != match;""")
TABLE_ROW = Template("""('${haystack}', '${needle}', ${match}, '${func}')""")
rows, table_queries, const_queries = create_cases(TABLE_ROW, TABLE_QUERY, CONST_QUERY)
rows, table_queries, const_queries = create_cases('hasToken', 'hasTokenCaseInsensitive', TABLE_ROW, TABLE_QUERY, CONST_QUERY)
for q in const_queries:
query(q)
@ -112,7 +145,8 @@ def main():
(
haystack String,
needle String,
match UInt8
match UInt8,
func String
)
ENGINE MergeTree()
ORDER BY haystack;
@ -120,5 +154,7 @@ INSERT INTO ht VALUES {values};""".format(values=", ".join(rows)))
for q in sorted(table_queries):
query(q)
query("""DROP TABLE ht""")
if __name__ == '__main__':
main()

View File

@ -11,6 +11,47 @@
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
0 expecting 0
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1
1 expecting 1

View File

@ -67,22 +67,22 @@ The command-line client allows passing external data (external temporary tables)
### Queries with Parameters {#cli-queries-with-parameters}
You can create a query with parameters, and pass values for these parameters with the parameters of the client app. For example:
You can create a query with parameters and pass values to them from client application. This allows to avoid formatting query with specific dynamic values on client side. For example:
```bash
clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}"
```
#### Syntax of a Query {#cli-queries-with-parameters-syntax}
#### Query Syntax {#cli-queries-with-parameters-syntax}
Format a query by the standard method. Values that you want to put into the query from the app parameters place in braces and format as follows:
Format a query as usual, then place the values that you want to pass from the app parameters to the query in braces in the following format:
```
{<name>:<data type>}
```
- `name`Identifier of a placeholder that should be used in app parameter as `--param_name = value`.
- `data type`A data type of app parameter value. For example, data structure like `(integer, ('string', integer))` can have a data type `Tuple(UInt8, Tuple(String, UInt8))` (also you can use another [integer](../data_types/int_uint.md) types).
- `name`Placeholder identifier. In the console client it should be used in app parameters as `--param_<name> = value`.
- `data type`[Data type](../data_types/index.md) of the app parameter value. For example, a data structure like `(integer, ('string', integer))` can have the `Tuple(UInt8, Tuple(String, UInt8))` data type (you can also use another [integer](../data_types/int_uint.md) types).
#### Example
@ -118,6 +118,8 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
- `--stacktrace` If specified, also print the stack trace if an exception occurs.
- `--config-file` The name of the configuration file.
- `--secure` If specified, will connect to server over secure connection.
- `--param_<name>` — Value for a [query with parameters](#cli-queries-with-parameters).
### Configuration Files

View File

@ -246,7 +246,7 @@ Use buffering to avoid situations where a query processing error occurred after
### Queries with Parameters {#cli-queries-with-parameters}
You can create a query with parameters, and pass values for these parameters with the parameters of the HTTP request. For more information, see [CLI Formatted Queries](cli.md#cli-queries-with-parameters).
You can create a query with parameters and pass values for them from the corresponding HTTP request parameters. For more information, see [Queries with Parameters for CLI](cli.md#cli-queries-with-parameters).
### Example

View File

@ -54,8 +54,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA
По умолчанию, в качестве формата, используется формат PrettyCompact (красивые таблички). Вы можете изменить формат с помощью секции FORMAT запроса, или с помощью указания `\G` на конце запроса, с помощью аргумента командной строки `--format` или `--vertical`, или с помощью конфигурационного файла клиента.
Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из:
"exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй"
Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из: "exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй"
При выполнении запроса, клиент показывает:
@ -68,38 +67,64 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA
Клиент командной строки позволяет передать внешние данные (внешние временные таблицы) для использования запроса. Подробнее смотрите раздел "Внешние данные для обработки запроса"
### Запросы с параметрами {#cli-queries-with-parameters}
Вы можете создать запрос с параметрами и передавать в них значения из приложения. Это позволяет избежать форматирования запросов на стороне клиента, если известно, какие из параметров запроса динамически меняются. Например:
```bash
clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}"
```
#### Cинтаксис запроса {#cli-queries-with-parameters-syntax}
Отформатируйте запрос обычным способом. Представьте значения, которые вы хотите передать из параметров приложения в запрос в следующем формате:
```
{<name>:<data type>}
```
- `name` — идентификатор подстановки. В консольном клиенте его следует использовать как часть имени параметра `--param_<name> = value`.
- `data type` — [тип данных](../data_types/index.md) значения. Например, структура данных `(integer, ('string', integer))` может иметь тип данных `Tuple(UInt8, Tuple(String, UInt8))` ([целочисленный](../data_types/int_uint.md) тип может быть и другим).
#### Пример
```bash
clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}"
```
## Конфигурирование {#interfaces_cli_configuration}
В `clickhouse-client` можно передавать различные параметры (все параметры имеют значения по умолчанию) с помощью:
- Командной строки.
Параметры командной строки переопределяют значения по умолчанию и параметры конфигурационных файлов.
- Конфигурационных файлов.
- Командной строки.
Параметры командной строки переопределяют значения по умолчанию и параметры конфигурационных файлов.
- Конфигурационных файлов.
Параметры в конфигурационных файлах переопределяют значения по умолчанию.
### Параметры командной строки
- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес.
- `--port` - порт, к которому соединяться, по умолчанию - 9000. Замечу, что для HTTP и родного интерфейса используются разные порты.
- `--user, -u` - имя пользователя, по умолчанию - default.
- `--password` - пароль, по умолчанию - пустая строка.
- `--query, -q` - запрос для выполнения, при использовании в неинтерактивном режиме.
- `--database, -d` - выбрать текущую БД, по умолчанию - текущая БД из настроек сервера (по умолчанию - БД default).
- `--multiline, -m` - если указано - разрешить многострочные запросы, не отправлять запрос по нажатию Enter.
- `--multiquery, -n` - если указано - разрешить выполнять несколько запросов, разделённых точкой с запятой. Работает только в неинтерактивном режиме.
- `--format, -f` - использовать указанный формат по умолчанию для вывода результата.
- `--vertical, -E` - если указано, использовать формат Vertical по умолчанию для вывода результата. То же самое, что --format=Vertical. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц.
- `--time, -t` - если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr.
- `--stacktrace` - если указано, в случае исключения, выводить также его стек трейс.
- `--config-file` - имя конфигурационного файла.
- `--secure` - если указано, будет использован безопасный канал.
- `--host, -h` — имя сервера, по умолчанию — localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес.
- `--port` — порт, к которому соединяться, по умолчанию — 9000. Замечу, что для HTTP и родного интерфейса используются разные порты.
- `--user, -u` — имя пользователя, по умолчанию — default.
- `--password` — пароль, по умолчанию — пустая строка.
- `--query, -q` — запрос для выполнения, при использовании в неинтерактивном режиме.
- `--database, -d` — выбрать текущую БД, по умолчанию — текущая БД из настроек сервера (по умолчанию — БД default).
- `--multiline, -m` — если указано — разрешить многострочные запросы, не отправлять запрос по нажатию Enter.
- `--multiquery, -n` — если указано — разрешить выполнять несколько запросов, разделённых точкой с запятой. Работает только в неинтерактивном режиме.
- `--format, -f` — использовать указанный формат по умолчанию для вывода результата.
- `--vertical, -E` — если указано, использовать формат Vertical по умолчанию для вывода результата. То же самое, что --format=Vertical. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц.
- `--time, -t` — если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr.
- `--stacktrace` — если указано, в случае исключения, выводить также его стек трейс.
- `--config-file` — имя конфигурационного файла.
- `--secure` — если указано, будет использован безопасный канал.
- `--param_<name>` — значение параметра для [запроса с параметрами](#cli-queries-with-parameters).
### Конфигурационные файлы
`clickhouse-client` использует первый существующий файл из:
`clickhouseclient` использует первый существующий файл из:
- Определенного параметром `--config-file`.
- `./clickhouse-client.xml`

View File

@ -245,5 +245,15 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa
Буферизация позволяет избежать ситуации когда код ответа и HTTP-заголовки были отправлены клиенту, после чего возникла ошибка выполнения запроса. В такой ситуации сообщение об ошибке записывается в конце тела ответа, и на стороне клиента ошибка может быть обнаружена только на этапе парсинга.
### Запросы с параметрами {#cli-queries-with-parameters}
Можно создать запрос с параметрами и передать для них значения из соответствующих параметров HTTP-запроса. Дополнительную информацию смотрите в [Запросы с параметрами для консольного клиента](cli.md#cli-queries-with-parameters).
### Пример
```bash
curl -sS "<address>?param_id=2¶m_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}"
```
[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/http_interface/) <!--hide-->

Some files were not shown because too many files have changed in this diff Show More