mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 12:32:04 +00:00
More general
This commit is contained in:
parent
5177309001
commit
13e4581317
@ -1,6 +1,8 @@
|
|||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
|
|
||||||
#include <Storages/AlterCommands.h>
|
#include <Storages/AlterCommands.h>
|
||||||
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
#include <Parsers/ASTSetQuery.h>
|
||||||
|
|
||||||
#include <sparsehash/dense_hash_map>
|
#include <sparsehash/dense_hash_map>
|
||||||
#include <sparsehash/dense_hash_set>
|
#include <sparsehash/dense_hash_set>
|
||||||
@ -19,6 +21,7 @@ namespace ErrorCodes
|
|||||||
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
||||||
extern const int TYPE_MISMATCH;
|
extern const int TYPE_MISMATCH;
|
||||||
extern const int SETTINGS_ARE_NOT_SUPPORTED;
|
extern const int SETTINGS_ARE_NOT_SUPPORTED;
|
||||||
|
extern const int UNKNOWN_SETTING;
|
||||||
}
|
}
|
||||||
|
|
||||||
IStorage::IStorage(ColumnsDescription columns_)
|
IStorage::IStorage(ColumnsDescription columns_)
|
||||||
@ -295,7 +298,9 @@ bool IStorage::isVirtualColumn(const String & column_name) const
|
|||||||
|
|
||||||
bool IStorage::hasSetting(const String & /* setting_name */) const
|
bool IStorage::hasSetting(const String & /* setting_name */) const
|
||||||
{
|
{
|
||||||
throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED);
|
if (!supportsSettings())
|
||||||
|
throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED);
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id)
|
TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id)
|
||||||
@ -352,6 +357,39 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
|
|||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void IStorage::alterSettings(
|
||||||
|
const SettingsChanges & new_changes,
|
||||||
|
const String & current_database_name,
|
||||||
|
const String & current_table_name,
|
||||||
|
const Context & context,
|
||||||
|
TableStructureWriteLockHolder & /* table_lock_holder */)
|
||||||
|
{
|
||||||
|
IDatabase::ASTModifier storage_modifier = [&] (IAST & ast)
|
||||||
|
{
|
||||||
|
if (!new_changes.empty())
|
||||||
|
{
|
||||||
|
auto & storage_changes = ast.as<ASTStorage &>().settings->changes;
|
||||||
|
/// Make storage settings unique
|
||||||
|
for (const auto & change : new_changes)
|
||||||
|
{
|
||||||
|
if (hasSetting(change.name))
|
||||||
|
{
|
||||||
|
auto finder = [&change] (const SettingChange & c) { return c.name == change.name;};
|
||||||
|
if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end())
|
||||||
|
it->value = change.value;
|
||||||
|
else
|
||||||
|
storage_changes.push_back(change);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), storage_modifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void IStorage::alter(
|
void IStorage::alter(
|
||||||
const AlterCommands & params,
|
const AlterCommands & params,
|
||||||
const String & database_name,
|
const String & database_name,
|
||||||
@ -359,12 +397,17 @@ void IStorage::alter(
|
|||||||
const Context & context,
|
const Context & context,
|
||||||
TableStructureWriteLockHolder & table_lock_holder)
|
TableStructureWriteLockHolder & table_lock_holder)
|
||||||
{
|
{
|
||||||
for (const auto & param : params)
|
if (params.isSettingsAlter())
|
||||||
{
|
{
|
||||||
if (param.isMutable())
|
SettingsChanges new_changes;
|
||||||
throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
params.applyForSettingsOnly(new_changes);
|
||||||
|
alterSettings(new_changes, database_name, table_name, context, table_lock_holder);
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (params.isMutable())
|
||||||
|
throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
|
||||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||||
auto new_columns = getColumns();
|
auto new_columns = getColumns();
|
||||||
auto new_indices = getIndices();
|
auto new_indices = getIndices();
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include <Common/ActionLock.h>
|
#include <Common/ActionLock.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/RWLock.h>
|
#include <Common/RWLock.h>
|
||||||
|
#include <Common/SettingsChanges.h>
|
||||||
|
|
||||||
#include <optional>
|
#include <optional>
|
||||||
#include <shared_mutex>
|
#include <shared_mutex>
|
||||||
@ -95,6 +96,9 @@ public:
|
|||||||
/// Returns true if the storage supports deduplication of inserted data blocks.
|
/// Returns true if the storage supports deduplication of inserted data blocks.
|
||||||
virtual bool supportsDeduplication() const { return false; }
|
virtual bool supportsDeduplication() const { return false; }
|
||||||
|
|
||||||
|
/// Returns true if the storage supports settings.
|
||||||
|
virtual bool supportsSettings() const { return false; }
|
||||||
|
|
||||||
/// Optional size information of each physical column.
|
/// Optional size information of each physical column.
|
||||||
/// Currently it's only used by the MergeTree family for query optimizations.
|
/// Currently it's only used by the MergeTree family for query optimizations.
|
||||||
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||||
@ -252,6 +256,15 @@ public:
|
|||||||
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** ALTER table settings if possible. Otherwise throws exception.
|
||||||
|
*/
|
||||||
|
virtual void alterSettings(
|
||||||
|
const SettingsChanges & new_changes,
|
||||||
|
const String & current_database_name,
|
||||||
|
const String & current_table_name,
|
||||||
|
const Context & context,
|
||||||
|
TableStructureWriteLockHolder & table_lock_holder);
|
||||||
|
|
||||||
/** Perform any background work. For example, combining parts in a MergeTree type table.
|
/** Perform any background work. For example, combining parts in a MergeTree type table.
|
||||||
* Returns whether any work has been done.
|
* Returns whether any work has been done.
|
||||||
*/
|
*/
|
||||||
|
@ -40,6 +40,7 @@ namespace ErrorCodes
|
|||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int UNSUPPORTED_METHOD;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
@ -388,6 +389,22 @@ bool StorageKafka::streamToViews()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool StorageKafka::hasSetting(const String & setting_name) const
|
||||||
|
{
|
||||||
|
return KafkaSettings::findIndex(setting_name) != KafkaSettings::npos;
|
||||||
|
}
|
||||||
|
|
||||||
|
void StorageKafka::alterSettings(
|
||||||
|
const SettingsChanges & /* new_changes */,
|
||||||
|
const String & /* current_database_name */,
|
||||||
|
const String & /* current_table_name */,
|
||||||
|
const Context & /* context */,
|
||||||
|
TableStructureWriteLockHolder & /* table_lock_holder */)
|
||||||
|
{
|
||||||
|
throw Exception("Storage '" + getName() + "' doesn't support settings alter", ErrorCodes::UNSUPPORTED_METHOD);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void registerStorageKafka(StorageFactory & factory)
|
void registerStorageKafka(StorageFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args)
|
factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args)
|
||||||
|
@ -24,6 +24,7 @@ public:
|
|||||||
std::string getName() const override { return "Kafka"; }
|
std::string getName() const override { return "Kafka"; }
|
||||||
std::string getTableName() const override { return table_name; }
|
std::string getTableName() const override { return table_name; }
|
||||||
std::string getDatabaseName() const override { return database_name; }
|
std::string getDatabaseName() const override { return database_name; }
|
||||||
|
bool supportsSettings() const override { return true; }
|
||||||
|
|
||||||
void startup() override;
|
void startup() override;
|
||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
@ -50,6 +51,15 @@ public:
|
|||||||
const auto & getSchemaName() const { return schema_name; }
|
const auto & getSchemaName() const { return schema_name; }
|
||||||
const auto & skipBroken() const { return skip_broken; }
|
const auto & skipBroken() const { return skip_broken; }
|
||||||
|
|
||||||
|
bool hasSetting(const String & setting_name) const override;
|
||||||
|
|
||||||
|
void alterSettings(
|
||||||
|
const SettingsChanges & new_changes,
|
||||||
|
const String & current_database_name,
|
||||||
|
const String & current_table_name,
|
||||||
|
const Context & context,
|
||||||
|
TableStructureWriteLockHolder & table_lock_holder) override;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageKafka(
|
StorageKafka(
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
|
@ -1588,26 +1588,10 @@ void MergeTreeData::alterSettings(
|
|||||||
const String & current_database_name,
|
const String & current_database_name,
|
||||||
const String & current_table_name,
|
const String & current_table_name,
|
||||||
const Context & context,
|
const Context & context,
|
||||||
TableStructureWriteLockHolder & /* table_lock_holder */)
|
TableStructureWriteLockHolder & table_lock_holder)
|
||||||
{
|
{
|
||||||
settings.updateFromChanges(new_changes);
|
settings.updateFromChanges(new_changes);
|
||||||
IDatabase::ASTModifier storage_modifier = [&] (IAST & ast)
|
IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder);
|
||||||
{
|
|
||||||
if (!new_changes.empty())
|
|
||||||
{
|
|
||||||
auto & storage_changes = ast.as<ASTStorage &>().settings->changes;
|
|
||||||
/// Make storage settings unique
|
|
||||||
for (const auto & change : new_changes)
|
|
||||||
{
|
|
||||||
auto finder = [&change] (const SettingChange & c) { return c.name == change.name;};
|
|
||||||
if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end())
|
|
||||||
it->value = change.value;
|
|
||||||
else
|
|
||||||
storage_changes.push_back(change);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, getColumns(), getIndices(), storage_modifier);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTreeData::hasSetting(const String & setting_name) const
|
bool MergeTreeData::hasSetting(const String & setting_name) const
|
||||||
@ -2245,7 +2229,7 @@ std::optional<Int64> MergeTreeData::getMinPartDataVersion() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event *until) const
|
void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const
|
||||||
{
|
{
|
||||||
const size_t parts_count_in_total = getPartsCount();
|
const size_t parts_count_in_total = getPartsCount();
|
||||||
if (parts_count_in_total >= settings.max_parts_in_total)
|
if (parts_count_in_total >= settings.max_parts_in_total)
|
||||||
|
@ -342,6 +342,8 @@ public:
|
|||||||
|| merging_params.mode == MergingParams::VersionedCollapsing;
|
|| merging_params.mode == MergingParams::VersionedCollapsing;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool supportsSettings() const override { return true; }
|
||||||
|
|
||||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const override;
|
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const override;
|
||||||
|
|
||||||
NameAndTypePair getColumn(const String & column_name) const override
|
NameAndTypePair getColumn(const String & column_name) const override
|
||||||
@ -516,7 +518,7 @@ public:
|
|||||||
const String & current_database_name,
|
const String & current_database_name,
|
||||||
const String & current_table_name,
|
const String & current_table_name,
|
||||||
const Context & context,
|
const Context & context,
|
||||||
TableStructureWriteLockHolder & table_lock_holder);
|
TableStructureWriteLockHolder & table_lock_holder) override;
|
||||||
|
|
||||||
/// All MergeTreeData children have settings.
|
/// All MergeTreeData children have settings.
|
||||||
bool hasSetting(const String & setting_name) const override;
|
bool hasSetting(const String & setting_name) const override;
|
||||||
|
Loading…
Reference in New Issue
Block a user