未验证 提交 957d2326 编写于 作者: I Ivan 提交者: GitHub

Minimal implementation of row-level security CLICKHOUSE-4315 (#4792)

For detailed description see the related PR
上级 5588618c
...@@ -74,6 +74,26 @@ ...@@ -74,6 +74,26 @@
<!-- Quota for user. --> <!-- Quota for user. -->
<quota>default</quota> <quota>default</quota>
<!-- For testing the table filters -->
<databases>
<test>
<!-- Simple expression filter -->
<filtered_table1>
<filter>a = 1</filter>
</filtered_table1>
<!-- Complex expression filter -->
<filtered_table2>
<filter>a + b &lt; 1 or c - d &gt; 5</filter>
</filtered_table2>
<!-- Filter with ALIAS column -->
<filtered_table3>
<filter>c = 1</filter>
</filtered_table3>
</test>
</databases>
</default> </default>
<!-- Example of user with readonly access. --> <!-- Example of user with readonly access. -->
......
#include <Interpreters/ProcessList.h>
#include <DataStreams/BlockIO.h>
namespace DB
{
BlockIO::~BlockIO() = default;
BlockIO::BlockIO() = default;
BlockIO::BlockIO(const BlockIO &) = default;
}
...@@ -11,15 +11,19 @@ class ProcessListEntry; ...@@ -11,15 +11,19 @@ class ProcessListEntry;
struct BlockIO struct BlockIO
{ {
BlockIO() = default;
BlockIO(const BlockIO &) = default;
~BlockIO() = default;
BlockOutputStreamPtr out;
BlockInputStreamPtr in;
/** process_list_entry should be destroyed after in and after out, /** process_list_entry should be destroyed after in and after out,
* since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example), * since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example),
* which could be used before destroying of in and out. * which could be used before destroying of in and out.
*/ */
std::shared_ptr<ProcessListEntry> process_list_entry; std::shared_ptr<ProcessListEntry> process_list_entry;
BlockInputStreamPtr in;
BlockOutputStreamPtr out;
/// Callbacks for query logging could be set here. /// Callbacks for query logging could be set here.
std::function<void(IBlockInputStream *, IBlockOutputStream *)> finish_callback; std::function<void(IBlockInputStream *, IBlockOutputStream *)> finish_callback;
std::function<void()> exception_callback; std::function<void()> exception_callback;
...@@ -37,17 +41,11 @@ struct BlockIO ...@@ -37,17 +41,11 @@ struct BlockIO
exception_callback(); exception_callback();
} }
/// We provide the correct order of destruction. BlockIO & operator= (const BlockIO & rhs)
void reset()
{ {
out.reset(); out.reset();
in.reset(); in.reset();
process_list_entry.reset(); process_list_entry.reset();
}
BlockIO & operator= (const BlockIO & rhs)
{
reset();
process_list_entry = rhs.process_list_entry; process_list_entry = rhs.process_list_entry;
in = rhs.in; in = rhs.in;
...@@ -58,10 +56,6 @@ struct BlockIO ...@@ -58,10 +56,6 @@ struct BlockIO
return *this; return *this;
} }
~BlockIO();
BlockIO();
BlockIO(const BlockIO &);
}; };
} }
...@@ -26,7 +26,7 @@ ...@@ -26,7 +26,7 @@
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Interpreters/ExpressionJIT.h> #include <Interpreters/ExpressionJIT.h>
#include <Interpreters/RuntimeComponentsFactory.h> #include <Interpreters/RuntimeComponentsFactory.h>
#include <Interpreters/ISecurityManager.h> #include <Interpreters/IUsersManager.h>
#include <Interpreters/Quota.h> #include <Interpreters/Quota.h>
#include <Interpreters/EmbeddedDictionaries.h> #include <Interpreters/EmbeddedDictionaries.h>
#include <Interpreters/ExternalDictionaries.h> #include <Interpreters/ExternalDictionaries.h>
...@@ -129,7 +129,7 @@ struct ContextShared ...@@ -129,7 +129,7 @@ struct ContextShared
mutable std::optional<ExternalModels> external_models; mutable std::optional<ExternalModels> external_models;
String default_profile_name; /// Default profile name used for default values. String default_profile_name; /// Default profile name used for default values.
String system_profile_name; /// Profile used by system processes String system_profile_name; /// Profile used by system processes
std::unique_ptr<ISecurityManager> security_manager; /// Known users. std::unique_ptr<IUsersManager> users_manager; /// Known users.
Quotas quotas; /// Known quotas for resource use. Quotas quotas; /// Known quotas for resource use.
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
...@@ -291,7 +291,7 @@ struct ContextShared ...@@ -291,7 +291,7 @@ struct ContextShared
private: private:
void initialize() void initialize()
{ {
security_manager = runtime_components_factory->createSecurityManager(); users_manager = runtime_components_factory->createUsersManager();
} }
}; };
...@@ -571,7 +571,7 @@ void Context::setUsersConfig(const ConfigurationPtr & config) ...@@ -571,7 +571,7 @@ void Context::setUsersConfig(const ConfigurationPtr & config)
{ {
auto lock = getLock(); auto lock = getLock();
shared->users_config = config; shared->users_config = config;
shared->security_manager->loadFromConfig(*shared->users_config); shared->users_manager->loadFromConfig(*shared->users_config);
shared->quotas.loadFromConfig(*shared->users_config); shared->quotas.loadFromConfig(*shared->users_config);
} }
...@@ -581,11 +581,39 @@ ConfigurationPtr Context::getUsersConfig() ...@@ -581,11 +581,39 @@ ConfigurationPtr Context::getUsersConfig()
return shared->users_config; return shared->users_config;
} }
bool Context::hasUserProperty(const String & database, const String & table, const String & name) const
{
auto lock = getLock();
// No user - no properties.
if (client_info.current_user.empty())
return false;
const auto & props = shared->users_manager->getUser(client_info.current_user)->table_props;
auto db = props.find(database);
if (db == props.end())
return false;
auto table_props = db->second.find(table);
if (table_props == db->second.end())
return false;
return !!table_props->second.count(name);
}
const String & Context::getUserProperty(const String & database, const String & table, const String & name) const
{
auto lock = getLock();
const auto & props = shared->users_manager->getUser(client_info.current_user)->table_props;
return props.at(database).at(table).at(name);
}
void Context::calculateUserSettings() void Context::calculateUserSettings()
{ {
auto lock = getLock(); auto lock = getLock();
String profile = shared->security_manager->getUser(client_info.current_user)->profile; String profile = shared->users_manager->getUser(client_info.current_user)->profile;
/// 1) Set default settings (hardcoded values) /// 1) Set default settings (hardcoded values)
/// NOTE: we ignore global_context settings (from which it is usually copied) /// NOTE: we ignore global_context settings (from which it is usually copied)
...@@ -606,7 +634,7 @@ void Context::setUser(const String & name, const String & password, const Poco:: ...@@ -606,7 +634,7 @@ void Context::setUser(const String & name, const String & password, const Poco::
{ {
auto lock = getLock(); auto lock = getLock();
auto user_props = shared->security_manager->authorizeAndGetUser(name, password, address.host()); auto user_props = shared->users_manager->authorizeAndGetUser(name, password, address.host());
client_info.current_user = name; client_info.current_user = name;
client_info.current_address = address; client_info.current_address = address;
...@@ -644,7 +672,7 @@ bool Context::hasDatabaseAccessRights(const String & database_name) const ...@@ -644,7 +672,7 @@ bool Context::hasDatabaseAccessRights(const String & database_name) const
{ {
auto lock = getLock(); auto lock = getLock();
return client_info.current_user.empty() || (database_name == "system") || return client_info.current_user.empty() || (database_name == "system") ||
shared->security_manager->hasAccessToDatabase(client_info.current_user, database_name); shared->users_manager->hasAccessToDatabase(client_info.current_user, database_name);
} }
void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const
...@@ -655,7 +683,7 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c ...@@ -655,7 +683,7 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c
/// All users have access to the database system. /// All users have access to the database system.
return; return;
} }
if (!shared->security_manager->hasAccessToDatabase(client_info.current_user, database_name)) if (!shared->users_manager->hasAccessToDatabase(client_info.current_user, database_name))
throw Exception("Access denied to database " + database_name + " for user " + client_info.current_user , ErrorCodes::DATABASE_ACCESS_DENIED); throw Exception("Access denied to database " + database_name + " for user " + client_info.current_user , ErrorCodes::DATABASE_ACCESS_DENIED);
} }
......
...@@ -188,6 +188,10 @@ public: ...@@ -188,6 +188,10 @@ public:
void setUsersConfig(const ConfigurationPtr & config); void setUsersConfig(const ConfigurationPtr & config);
ConfigurationPtr getUsersConfig(); ConfigurationPtr getUsersConfig();
// User property is a key-value pair from the configuration entry: users.<username>.databases.<db_name>.<table_name>.<key_name>
bool hasUserProperty(const String & database, const String & table, const String & name) const;
const String & getUserProperty(const String & database, const String & table, const String & name) const;
/// Must be called before getClientInfo. /// Must be called before getClientInfo.
void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key); void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key);
/// Compute and set actual user settings, client_info.current_user should be set /// Compute and set actual user settings, client_info.current_user should be set
......
...@@ -2,7 +2,7 @@ ...@@ -2,7 +2,7 @@
#include <Dictionaries/Embedded/IGeoDictionariesLoader.h> #include <Dictionaries/Embedded/IGeoDictionariesLoader.h>
#include <Interpreters/IExternalLoaderConfigRepository.h> #include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Interpreters/ISecurityManager.h> #include <Interpreters/IUsersManager.h>
#include <memory> #include <memory>
...@@ -16,7 +16,9 @@ namespace DB ...@@ -16,7 +16,9 @@ namespace DB
class IRuntimeComponentsFactory class IRuntimeComponentsFactory
{ {
public: public:
virtual std::unique_ptr<ISecurityManager> createSecurityManager() = 0; virtual ~IRuntimeComponentsFactory() = default;
virtual std::unique_ptr<IUsersManager> createUsersManager() = 0;
virtual std::unique_ptr<IGeoDictionariesLoader> createGeoDictionariesLoader() = 0; virtual std::unique_ptr<IGeoDictionariesLoader> createGeoDictionariesLoader() = 0;
...@@ -24,8 +26,6 @@ public: ...@@ -24,8 +26,6 @@ public:
virtual std::unique_ptr<IExternalLoaderConfigRepository> createExternalDictionariesConfigRepository() = 0; virtual std::unique_ptr<IExternalLoaderConfigRepository> createExternalDictionariesConfigRepository() = 0;
virtual std::unique_ptr<IExternalLoaderConfigRepository> createExternalModelsConfigRepository() = 0; virtual std::unique_ptr<IExternalLoaderConfigRepository> createExternalModelsConfigRepository() = 0;
virtual ~IRuntimeComponentsFactory() {}
}; };
} }
...@@ -5,16 +5,18 @@ ...@@ -5,16 +5,18 @@
namespace DB namespace DB
{ {
/** Duties of security manager: /** Duties of users manager:
* 1) Authenticate users * 1) Authenticate users
* 2) Provide user settings (profile, quota, ACLs) * 2) Provide user settings (profile, quota, ACLs)
* 3) Grant access to databases * 3) Grant access to databases
*/ */
class ISecurityManager class IUsersManager
{ {
public: public:
using UserPtr = std::shared_ptr<const User>; using UserPtr = std::shared_ptr<const User>;
virtual ~IUsersManager() = default;
virtual void loadFromConfig(const Poco::Util::AbstractConfiguration & config) = 0; virtual void loadFromConfig(const Poco::Util::AbstractConfiguration & config) = 0;
/// Find user and make authorize checks /// Find user and make authorize checks
...@@ -28,8 +30,6 @@ public: ...@@ -28,8 +30,6 @@ public:
/// Check if the user has access to the database. /// Check if the user has access to the database.
virtual bool hasAccessToDatabase(const String & user_name, const String & database_name) const = 0; virtual bool hasAccessToDatabase(const String & user_name, const String & database_name) const = 0;
virtual ~ISecurityManager() {}
}; };
} }
...@@ -104,13 +104,13 @@ private: ...@@ -104,13 +104,13 @@ private:
BlockInputStreamPtr & firstStream() { return streams.at(0); } BlockInputStreamPtr & firstStream() { return streams.at(0); }
template <typename Transform> template <typename Transform>
void transform(Transform && transform) void transform(Transform && transformation)
{ {
for (auto & stream : streams) for (auto & stream : streams)
transform(stream); transformation(stream);
if (stream_with_non_joined_data) if (stream_with_non_joined_data)
transform(stream_with_non_joined_data); transformation(stream_with_non_joined_data);
} }
bool hasMoreThanOneStream() const bool hasMoreThanOneStream() const
...@@ -154,9 +154,10 @@ private: ...@@ -154,9 +154,10 @@ private:
SubqueriesForSets subqueries_for_sets; SubqueriesForSets subqueries_for_sets;
PrewhereInfoPtr prewhere_info; PrewhereInfoPtr prewhere_info;
FilterInfoPtr filter_info;
}; };
AnalysisResult analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run); AnalysisResult analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run, const FilterInfoPtr & filter_info);
/** From which table to read. With JOIN, the "left" table is returned. /** From which table to read. With JOIN, the "left" table is returned.
......
#pragma once #pragma once
#include <Dictionaries/Embedded/GeoDictionariesLoader.h> #include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/IRuntimeComponentsFactory.h>
#include <Interpreters/ExternalLoaderConfigRepository.h> #include <Interpreters/ExternalLoaderConfigRepository.h>
#include <Interpreters/SecurityManager.h> #include <Interpreters/IRuntimeComponentsFactory.h>
#include <Interpreters/UsersManager.h>
namespace DB namespace DB
{ {
...@@ -14,9 +14,9 @@ namespace DB ...@@ -14,9 +14,9 @@ namespace DB
class RuntimeComponentsFactory : public IRuntimeComponentsFactory class RuntimeComponentsFactory : public IRuntimeComponentsFactory
{ {
public: public:
std::unique_ptr<ISecurityManager> createSecurityManager() override std::unique_ptr<IUsersManager> createUsersManager() override
{ {
return std::make_unique<SecurityManager>(); return std::make_unique<UsersManager>();
} }
std::unique_ptr<IGeoDictionariesLoader> createGeoDictionariesLoader() override std::unique_ptr<IGeoDictionariesLoader> createGeoDictionariesLoader() override
......
...@@ -315,6 +315,34 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A ...@@ -315,6 +315,34 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A
databases.insert(database_name); databases.insert(database_name);
} }
} }
/// Read properties per "database.table"
/// Only tables are expected to have properties, so that all the keys inside "database" are table names.
const auto config_databases = config_elem + ".databases";
if (config.has(config_databases))
{
Poco::Util::AbstractConfiguration::Keys database_names;
config.keys(config_databases, database_names);
/// Read tables within databases
for (const auto & database : database_names)
{
const auto config_database = config_databases + "." + database;
Poco::Util::AbstractConfiguration::Keys table_names;
config.keys(config_database, table_names);
/// Read table properties
for (const auto & table : table_names)
{
const auto config_filter = config_database + "." + table + ".filter";
if (config.has(config_filter))
{
const auto filter_query = config.getString(config_filter);
table_props[database][table]["filter"] = filter_query;
}
}
}
}
} }
......
...@@ -2,9 +2,10 @@ ...@@ -2,9 +2,10 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <vector>
#include <unordered_set>
#include <memory> #include <memory>
#include <unordered_map>
#include <unordered_set>
#include <vector>
namespace Poco namespace Poco
...@@ -65,6 +66,12 @@ struct User ...@@ -65,6 +66,12 @@ struct User
using DatabaseSet = std::unordered_set<std::string>; using DatabaseSet = std::unordered_set<std::string>;
DatabaseSet databases; DatabaseSet databases;
/// Table properties.
using PropertyMap = std::unordered_map<std::string /* name */, std::string /* value */>;
using TableMap = std::unordered_map<std::string /* table */, PropertyMap /* properties */>;
using DatabaseMap = std::unordered_map<std::string /* database */, TableMap /* tables */>;
DatabaseMap table_props;
User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config); User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config);
}; };
......
#include "SecurityManager.h" #include <Interpreters/UsersManager.h>
#include <Poco/Net/IPAddress.h> #include <Poco/Net/IPAddress.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <Poco/String.h> #include <Poco/String.h>
...@@ -28,9 +29,9 @@ namespace ErrorCodes ...@@ -28,9 +29,9 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED; extern const int SUPPORT_IS_DISABLED;
} }
using UserPtr = SecurityManager::UserPtr; using UserPtr = UsersManager::UserPtr;
void SecurityManager::loadFromConfig(const Poco::Util::AbstractConfiguration & config) void UsersManager::loadFromConfig(const Poco::Util::AbstractConfiguration & config)
{ {
Container new_users; Container new_users;
...@@ -46,7 +47,7 @@ void SecurityManager::loadFromConfig(const Poco::Util::AbstractConfiguration & c ...@@ -46,7 +47,7 @@ void SecurityManager::loadFromConfig(const Poco::Util::AbstractConfiguration & c
users = std::move(new_users); users = std::move(new_users);
} }
UserPtr SecurityManager::authorizeAndGetUser( UserPtr UsersManager::authorizeAndGetUser(
const String & user_name, const String & user_name,
const String & password, const String & password,
const Poco::Net::IPAddress & address) const const Poco::Net::IPAddress & address) const
...@@ -100,7 +101,7 @@ UserPtr SecurityManager::authorizeAndGetUser( ...@@ -100,7 +101,7 @@ UserPtr SecurityManager::authorizeAndGetUser(
return it->second; return it->second;
} }
UserPtr SecurityManager::getUser(const String & user_name) const UserPtr UsersManager::getUser(const String & user_name) const
{ {
auto it = users.find(user_name); auto it = users.find(user_name);
...@@ -110,7 +111,7 @@ UserPtr SecurityManager::getUser(const String & user_name) const ...@@ -110,7 +111,7 @@ UserPtr SecurityManager::getUser(const String & user_name) const
return it->second; return it->second;
} }
bool SecurityManager::hasAccessToDatabase(const std::string & user_name, const std::string & database_name) const bool UsersManager::hasAccessToDatabase(const std::string & user_name, const std::string & database_name) const
{ {
auto it = users.find(user_name); auto it = users.find(user_name);
......
#pragma once #pragma once
#include <Interpreters/ISecurityManager.h> #include <Interpreters/IUsersManager.h>
#include <map> #include <map>
namespace DB namespace DB
{ {
/** Default implementation of security manager used by native server application. /** Default implementation of users manager used by native server application.
* Manages fixed set of users listed in 'Users' configuration file. * Manages fixed set of users listed in 'Users' configuration file.
*/ */
class SecurityManager : public ISecurityManager class UsersManager : public IUsersManager
{ {
private:
using Container = std::map<String, UserPtr>;
Container users;
public: public:
void loadFromConfig(const Poco::Util::AbstractConfiguration & config) override; void loadFromConfig(const Poco::Util::AbstractConfiguration & config) override;
...@@ -27,6 +23,10 @@ public: ...@@ -27,6 +23,10 @@ public:
UserPtr getUser(const String & user_name) const override; UserPtr getUser(const String & user_name) const override;
bool hasAccessToDatabase(const String & user_name, const String & database_name) const override; bool hasAccessToDatabase(const String & user_name, const String & database_name) const override;
private:
using Container = std::map<String, UserPtr>;
Container users;
}; };
} }
#include <Common/Config/ConfigProcessor.h> #include <Common/Config/ConfigProcessor.h>
#include <Interpreters/SecurityManager.h> #include <Interpreters/UsersManager.h>
#include <boost/filesystem.hpp> #include <boost/filesystem.hpp>
#include <vector> #include <vector>
#include <string> #include <string>
...@@ -197,11 +197,11 @@ void runOneTest(const TestDescriptor & test_descriptor) ...@@ -197,11 +197,11 @@ void runOneTest(const TestDescriptor & test_descriptor)
throw std::runtime_error(os.str()); throw std::runtime_error(os.str());
} }
DB::SecurityManager security_manager; DB::UsersManager users_manager;
try try
{ {
security_manager.loadFromConfig(*config); users_manager.loadFromConfig(*config);
} }
catch (const Poco::Exception & ex) catch (const Poco::Exception & ex)
{ {
...@@ -216,7 +216,7 @@ void runOneTest(const TestDescriptor & test_descriptor) ...@@ -216,7 +216,7 @@ void runOneTest(const TestDescriptor & test_descriptor)
try try
{ {
res = security_manager.hasAccessToDatabase(entry.user_name, entry.database_name); res = users_manager.hasAccessToDatabase(entry.user_name, entry.database_name);
} }
catch (const Poco::Exception &) catch (const Poco::Exception &)
{ {
......
...@@ -61,9 +61,9 @@ public: ...@@ -61,9 +61,9 @@ public:
/// The main name of the table type (for example, StorageMergeTree). /// The main name of the table type (for example, StorageMergeTree).
virtual std::string getName() const = 0; virtual std::string getName() const = 0;
/** The name of the table. /// The name of the table.
*/
virtual std::string getTableName() const = 0; virtual std::string getTableName() const = 0;
virtual std::string getDatabaseName() const { return {}; } // FIXME: should be abstract method.
/** Returns true if the storage receives data from a remote server or servers. */ /** Returns true if the storage receives data from a remote server or servers. */
virtual bool isRemote() const { return false; } virtual bool isRemote() const { return false; }
......
...@@ -27,7 +27,7 @@ friend class KafkaBlockOutputStream; ...@@ -27,7 +27,7 @@ friend class KafkaBlockOutputStream;
public: 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 { return database_name; } std::string getDatabaseName() const override { return database_name; }
void startup() override; void startup() override;
void shutdown() override; void shutdown() override;
......
...@@ -25,7 +25,16 @@ struct PrewhereInfo ...@@ -25,7 +25,16 @@ struct PrewhereInfo
: prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {}
}; };
/// Helper struct to store all the information about the filter expression.
struct FilterInfo
{
ExpressionActionsPtr actions;
String column_name;
bool do_remove_column = false;
};
using PrewhereInfoPtr = std::shared_ptr<PrewhereInfo>; using PrewhereInfoPtr = std::shared_ptr<PrewhereInfo>;
using FilterInfoPtr = std::shared_ptr<FilterInfo>;
struct SyntaxAnalyzerResult; struct SyntaxAnalyzerResult;
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>; using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
......
...@@ -27,12 +27,9 @@ public: ...@@ -27,12 +27,9 @@ public:
void shutdown() override; void shutdown() override;
~StorageMergeTree() override; ~StorageMergeTree() override;
std::string getName() const override std::string getName() const override { return data.merging_params.getModeName() + "MergeTree"; }
{
return data.merging_params.getModeName() + "MergeTree";
}
std::string getTableName() const override { return table_name; } std::string getTableName() const override { return table_name; }
std::string getDatabaseName() const override { return database_name; }
bool supportsSampling() const override { return data.supportsSampling(); } bool supportsSampling() const override { return data.supportsSampling(); }
bool supportsPrewhere() const override { return data.supportsPrewhere(); } bool supportsPrewhere() const override { return data.supportsPrewhere(); }
......
...@@ -79,12 +79,10 @@ public: ...@@ -79,12 +79,10 @@ public:
void shutdown() override; void shutdown() override;
~StorageReplicatedMergeTree() override; ~StorageReplicatedMergeTree() override;
std::string getName() const override std::string getName() const override { return "Replicated" + data.merging_params.getModeName() + "MergeTree"; }
{
return "Replicated" + data.merging_params.getModeName() + "MergeTree";
}
std::string getTableName() const override { return table_name; } std::string getTableName() const override { return table_name; }
std::string getDatabaseName() const override { return database_name; }
bool supportsSampling() const override { return data.supportsSampling(); } bool supportsSampling() const override { return data.supportsSampling(); }
bool supportsFinal() const override { return data.supportsFinal(); } bool supportsFinal() const override { return data.supportsFinal(); }
bool supportsPrewhere() const override { return data.supportsPrewhere(); } bool supportsPrewhere() const override { return data.supportsPrewhere(); }
......
-- PREWHERE should fail
1 0
1 1
0 0 0 0
0 0 6 0
0 1
1 0
1
1
0
1
1
1
1
1
0
1
1
0
1
1
1
1
0
1
1
1
1
1
1 0 1 1
1 1 1 1
1 1 1 0
DROP TABLE IF EXISTS filtered_table1;
DROP TABLE IF EXISTS filtered_table2;
DROP TABLE IF EXISTS filtered_table3;
-- Filter: a = 1, values: (1, 0), (1, 1)
CREATE TABLE test.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO test.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1);
-- Filter: a + b < 1 or c - d > 5, values: (0, 0, 0, 0), (0, 0, 6, 0)
CREATE TABLE test.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO test.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0);
-- Filter: c = 1, values: (0, 1), (1, 0)
CREATE TABLE test.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
INSERT INTO test.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1);
SELECT '-- PREWHERE should fail';
SELECT * FROM test.filtered_table1 PREWHERE 1; -- { serverError 182 }
SELECT * FROM test.filtered_table2 PREWHERE 1; -- { serverError 182 }
SELECT * FROM test.filtered_table3 PREWHERE 1; -- { serverError 182 }
SELECT * FROM test.filtered_table1;
SELECT * FROM test.filtered_table2;
SELECT * FROM test.filtered_table3;
SELECT a FROM test.filtered_table1;
SELECT b FROM test.filtered_table1;
SELECT a FROM test.filtered_table1 WHERE a = 1;
SELECT a = 1 FROM test.filtered_table1;
SELECT a FROM test.filtered_table3;
SELECT b FROM test.filtered_table3;
SELECT c FROM test.filtered_table3;
SELECT a + b FROM test.filtered_table3;
SELECT a FROM test.filtered_table3 WHERE c = 1;
SELECT c = 1 FROM test.filtered_table3;
SELECT a + b = 1 FROM test.filtered_table3;
SELECT * FROM test.filtered_table1 as t1 ANY LEFT JOIN test.filtered_table1 as t2 ON t1.a = t2.b;
SELECT * FROM test.filtered_table1 as t2 ANY RIGHT JOIN test.filtered_table1 as t1 ON t2.b = t1.a;
DROP TABLE test.filtered_table1;
DROP TABLE test.filtered_table2;
DROP TABLE test.filtered_table3;
DROP TABLE IF EXISTS test.test;
CREATE TABLE test.test (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE = MergeTree ORDER BY a;
SELECT b FROM test.test PREWHERE c = 1;
DROP TABLE test;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册