From d5b4b40b4e3a6265040462abe5c73334c56d5dd3 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 19 Dec 2019 04:46:20 +0100 Subject: [PATCH 01/66] Add parser --- dbms/src/Interpreters/InterpreterSystemQuery.cpp | 14 +++++++------- dbms/src/Parsers/ASTSystemQuery.cpp | 7 +++++-- dbms/src/Parsers/ASTSystemQuery.h | 10 ++++++++-- dbms/src/Parsers/ParserSystemQuery.cpp | 9 +++++++-- 4 files changed, 27 insertions(+), 13 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index c742ac37a5f..28bf7b08cc6 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -102,7 +102,7 @@ void startStopAction(Context & context, ASTSystemQuery & query, StorageActionBlo if (!query.target_table.empty()) { - String database = !query.target_database.empty() ? query.target_database : context.getCurrentDatabase(); + String database = !query.database.empty() ? query.database : context.getCurrentDatabase(); if (start) manager->remove(database, query.target_table, action_type); @@ -137,8 +137,8 @@ BlockIO InterpreterSystemQuery::execute() system_context.setSetting("profile", context.getSystemProfileName()); /// Make canonical query for simpler processing - if (!query.target_table.empty() && query.target_database.empty()) - query.target_database = context.getCurrentDatabase(); + if (!query.target_table.empty() && query.database.empty()) + query.database = context.getCurrentDatabase(); switch (query.type) { @@ -233,8 +233,8 @@ BlockIO InterpreterSystemQuery::execute() restartReplicas(system_context); break; case Type::RESTART_REPLICA: - if (!tryRestartReplica(query.target_database, query.target_table, system_context)) - throw Exception("There is no " + query.target_database + "." + query.target_table + " replicated table", + if (!tryRestartReplica(query.database, query.target_table, system_context)) + throw Exception("There is no " + query.database + "." + query.target_table + " replicated table", ErrorCodes::BAD_ARGUMENTS); break; case Type::FLUSH_LOGS: @@ -335,7 +335,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) { - String database_name = !query.target_database.empty() ? query.target_database : context.getCurrentDatabase(); + String database_name = !query.database.empty() ? query.database : context.getCurrentDatabase(); const String & table_name = query.target_table; StoragePtr table = context.getTable(database_name, table_name); @@ -358,7 +358,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) void InterpreterSystemQuery::flushDistributed(ASTSystemQuery & query) { - String database_name = !query.target_database.empty() ? query.target_database : context.getCurrentDatabase(); + String database_name = !query.database.empty() ? query.database : context.getCurrentDatabase(); String & table_name = query.target_table; if (auto storage_distributed = dynamic_cast(context.getTable(database_name, table_name).get())) diff --git a/dbms/src/Parsers/ASTSystemQuery.cpp b/dbms/src/Parsers/ASTSystemQuery.cpp index 4e7525bb176..0d096bb805f 100644 --- a/dbms/src/Parsers/ASTSystemQuery.cpp +++ b/dbms/src/Parsers/ASTSystemQuery.cpp @@ -97,9 +97,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, { settings.ostr << " "; - if (!target_database.empty()) + if (!database.empty()) { - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(target_database) + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(database) << (settings.hilite ? hilite_none : "") << "."; } @@ -107,6 +107,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, << (settings.hilite ? hilite_none : ""); }; + if (!cluster.empty()) + { + } if ( type == Type::STOP_MERGES || type == Type::START_MERGES || type == Type::STOP_TTL_MERGES diff --git a/dbms/src/Parsers/ASTSystemQuery.h b/dbms/src/Parsers/ASTSystemQuery.h index 77e8591a5f5..fa2ee1b0a51 100644 --- a/dbms/src/Parsers/ASTSystemQuery.h +++ b/dbms/src/Parsers/ASTSystemQuery.h @@ -1,13 +1,14 @@ #pragma once #include "config_core.h" +#include #include namespace DB { -class ASTSystemQuery : public IAST +class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster { public: @@ -55,13 +56,18 @@ public: Type type = Type::UNKNOWN; String target_dictionary; - String target_database; + String database; String target_table; String getID(char) const override { return "SYSTEM query"; } ASTPtr clone() const override { return std::make_shared(*this); } + ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override + { + return removeOnCluster(clone(), new_database); + } + protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/dbms/src/Parsers/ParserSystemQuery.cpp b/dbms/src/Parsers/ParserSystemQuery.cpp index 0a5bd1bf63e..b1047be6bfc 100644 --- a/dbms/src/Parsers/ParserSystemQuery.cpp +++ b/dbms/src/Parsers/ParserSystemQuery.cpp @@ -41,6 +41,11 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & switch (res->type) { case Type::RELOAD_DICTIONARY: + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, res->cluster, expected)) + return false; + } if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary)) return false; break; @@ -48,7 +53,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::RESTART_REPLICA: case Type::SYNC_REPLICA: case Type::FLUSH_DISTRIBUTED: - if (!parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table)) + if (!parseDatabaseAndTableName(pos, expected, res->database, res->target_table)) return false; break; @@ -66,7 +71,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::START_REPLICATION_QUEUES: case Type::STOP_DISTRIBUTED_SENDS: case Type::START_DISTRIBUTED_SENDS: - parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table); + parseDatabaseAndTableName(pos, expected, res->database, res->target_table); break; default: From 9e808e4a0d833d004f3d0807f9213fa5dac73418 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 19 Dec 2019 08:54:43 +0100 Subject: [PATCH 02/66] Let the command SYSTEM RELOAD DICTIONARY to use ON CLUSTER syntax --- dbms/src/Interpreters/InterpreterSystemQuery.cpp | 4 ++++ dbms/src/Parsers/ASTSystemQuery.cpp | 5 +++-- dbms/src/Parsers/ParserSystemQuery.cpp | 5 ++++- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 28bf7b08cc6..272b38c86ba 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -130,6 +131,9 @@ BlockIO InterpreterSystemQuery::execute() { auto & query = query_ptr->as(); + if (!query.cluster.empty()) + return executeDDLQueryOnCluster(query_ptr, context, {query.database}); + using Type = ASTSystemQuery::Type; /// Use global context with fresh system profile settings diff --git a/dbms/src/Parsers/ASTSystemQuery.cpp b/dbms/src/Parsers/ASTSystemQuery.cpp index 0d096bb805f..578d151967c 100644 --- a/dbms/src/Parsers/ASTSystemQuery.cpp +++ b/dbms/src/Parsers/ASTSystemQuery.cpp @@ -108,8 +108,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, }; if (!cluster.empty()) - { - } + formatOnCluster(settings); if ( type == Type::STOP_MERGES || type == Type::START_MERGES || type == Type::STOP_TTL_MERGES @@ -133,7 +132,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, print_database_table(); } else if (type == Type::RELOAD_DICTIONARY) + { settings.ostr << " " << backQuoteIfNeed(target_dictionary); + } } diff --git a/dbms/src/Parsers/ParserSystemQuery.cpp b/dbms/src/Parsers/ParserSystemQuery.cpp index b1047be6bfc..a56c576aa96 100644 --- a/dbms/src/Parsers/ParserSystemQuery.cpp +++ b/dbms/src/Parsers/ParserSystemQuery.cpp @@ -38,14 +38,17 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & if (!found) return false; + + String cluster_str; switch (res->type) { case Type::RELOAD_DICTIONARY: if (ParserKeyword{"ON"}.ignore(pos, expected)) { - if (!ASTQueryWithOnCluster::parse(pos, res->cluster, expected)) + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; } + res->cluster = cluster_str; if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary)) return false; break; From 1f5a11b384c0d3106e8d90545997cacde16d53e8 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 19 Dec 2019 09:01:57 +0100 Subject: [PATCH 03/66] Add test for ON CLUSTER on RELOAD DICTIONARY command --- dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py b/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py index 31ee90de472..d77ab2b842a 100644 --- a/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py +++ b/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py @@ -43,6 +43,9 @@ def test_dictionary_ddl_on_cluster(started_cluster): assert node.query("SELECT count() from sometbl") == "1\n" assert node.query("SELECT dictGetString('default.somedict', 'value', toUInt64({}))".format(num)) == node.name + '\n' + instance = started_cluster.instances['ch1'] + started_cluster.ddl_check_query(instance, "SYSTEM RELOAD DICTIONARY ON CLUSTER 'cluster' default.somedict") + ch1.query("DETACH DICTIONARY default.somedict ON CLUSTER 'cluster'") for node in [ch1, ch2, ch3, ch4]: From d5d8fd7004f461b1f15f8a78258b6b0e22798b71 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 19 Dec 2019 10:27:12 +0100 Subject: [PATCH 04/66] refactor target_table to table for better hemogny --- dbms/src/Interpreters/InterpreterSystemQuery.cpp | 16 ++++++++-------- dbms/src/Parsers/ASTSystemQuery.cpp | 4 ++-- dbms/src/Parsers/ASTSystemQuery.h | 2 +- dbms/src/Parsers/ParserSystemQuery.cpp | 4 ++-- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 272b38c86ba..d346ddd04df 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -101,14 +101,14 @@ void startStopAction(Context & context, ASTSystemQuery & query, StorageActionBlo auto manager = context.getActionLocksManager(); manager->cleanExpired(); - if (!query.target_table.empty()) + if (!query.table.empty()) { String database = !query.database.empty() ? query.database : context.getCurrentDatabase(); if (start) - manager->remove(database, query.target_table, action_type); + manager->remove(database, query.table, action_type); else - manager->add(database, query.target_table, action_type); + manager->add(database, query.table, action_type); } else { @@ -141,7 +141,7 @@ BlockIO InterpreterSystemQuery::execute() system_context.setSetting("profile", context.getSystemProfileName()); /// Make canonical query for simpler processing - if (!query.target_table.empty() && query.database.empty()) + if (!query.table.empty() && query.database.empty()) query.database = context.getCurrentDatabase(); switch (query.type) @@ -237,8 +237,8 @@ BlockIO InterpreterSystemQuery::execute() restartReplicas(system_context); break; case Type::RESTART_REPLICA: - if (!tryRestartReplica(query.database, query.target_table, system_context)) - throw Exception("There is no " + query.database + "." + query.target_table + " replicated table", + if (!tryRestartReplica(query.database, query.table, system_context)) + throw Exception("There is no " + query.database + "." + query.table + " replicated table", ErrorCodes::BAD_ARGUMENTS); break; case Type::FLUSH_LOGS: @@ -340,7 +340,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) { String database_name = !query.database.empty() ? query.database : context.getCurrentDatabase(); - const String & table_name = query.target_table; + const String & table_name = query.table; StoragePtr table = context.getTable(database_name, table_name); @@ -363,7 +363,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) void InterpreterSystemQuery::flushDistributed(ASTSystemQuery & query) { String database_name = !query.database.empty() ? query.database : context.getCurrentDatabase(); - String & table_name = query.target_table; + String & table_name = query.table; if (auto storage_distributed = dynamic_cast(context.getTable(database_name, table_name).get())) storage_distributed->flushClusterNodesAllData(); diff --git a/dbms/src/Parsers/ASTSystemQuery.cpp b/dbms/src/Parsers/ASTSystemQuery.cpp index 578d151967c..d1c10b8cfd7 100644 --- a/dbms/src/Parsers/ASTSystemQuery.cpp +++ b/dbms/src/Parsers/ASTSystemQuery.cpp @@ -103,7 +103,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, << (settings.hilite ? hilite_none : "") << "."; } - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(target_table) + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(table) << (settings.hilite ? hilite_none : ""); }; @@ -124,7 +124,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, || type == Type::STOP_DISTRIBUTED_SENDS || type == Type::START_DISTRIBUTED_SENDS) { - if (!target_table.empty()) + if (!table.empty()) print_database_table(); } else if (type == Type::RESTART_REPLICA || type == Type::SYNC_REPLICA || type == Type::FLUSH_DISTRIBUTED) diff --git a/dbms/src/Parsers/ASTSystemQuery.h b/dbms/src/Parsers/ASTSystemQuery.h index fa2ee1b0a51..f5b9afde4b3 100644 --- a/dbms/src/Parsers/ASTSystemQuery.h +++ b/dbms/src/Parsers/ASTSystemQuery.h @@ -57,7 +57,7 @@ public: String target_dictionary; String database; - String target_table; + String table; String getID(char) const override { return "SYSTEM query"; } diff --git a/dbms/src/Parsers/ParserSystemQuery.cpp b/dbms/src/Parsers/ParserSystemQuery.cpp index a56c576aa96..5bfc1713585 100644 --- a/dbms/src/Parsers/ParserSystemQuery.cpp +++ b/dbms/src/Parsers/ParserSystemQuery.cpp @@ -56,7 +56,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::RESTART_REPLICA: case Type::SYNC_REPLICA: case Type::FLUSH_DISTRIBUTED: - if (!parseDatabaseAndTableName(pos, expected, res->database, res->target_table)) + if (!parseDatabaseAndTableName(pos, expected, res->database, res->table)) return false; break; @@ -74,7 +74,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::START_REPLICATION_QUEUES: case Type::STOP_DISTRIBUTED_SENDS: case Type::START_DISTRIBUTED_SENDS: - parseDatabaseAndTableName(pos, expected, res->database, res->target_table); + parseDatabaseAndTableName(pos, expected, res->database, res->table); break; default: From abbbf3e72670ba2f578de3d89c92e0eaa7f3456d Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 19 Dec 2019 10:29:13 +0100 Subject: [PATCH 05/66] cosmetic --- dbms/src/Parsers/ASTSystemQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Parsers/ASTSystemQuery.cpp b/dbms/src/Parsers/ASTSystemQuery.cpp index d1c10b8cfd7..1e542305f00 100644 --- a/dbms/src/Parsers/ASTSystemQuery.cpp +++ b/dbms/src/Parsers/ASTSystemQuery.cpp @@ -132,9 +132,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, print_database_table(); } else if (type == Type::RELOAD_DICTIONARY) - { settings.ostr << " " << backQuoteIfNeed(target_dictionary); - } } From 9f5ab54fc2256efe44f9046cceabc1e79c019c6a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 17 Nov 2019 14:57:02 +0300 Subject: [PATCH 06/66] Rework RowPolicy based on IAccessEntity. --- dbms/src/Access/AccessControlManager.cpp | 11 +- dbms/src/Access/AccessControlManager.h | 5 + dbms/src/Access/IAccessEntity.cpp | 3 + dbms/src/Access/RowPolicy.cpp | 111 +++++++ dbms/src/Access/RowPolicy.h | 81 +++++ dbms/src/Access/RowPolicyContext.cpp | 59 ++++ dbms/src/Access/RowPolicyContext.h | 66 ++++ dbms/src/Access/RowPolicyContextFactory.cpp | 314 ++++++++++++++++++ dbms/src/Access/RowPolicyContextFactory.h | 54 +++ dbms/src/Access/UsersConfigAccessStorage.cpp | 56 ++++ dbms/src/Interpreters/Context.cpp | 31 +- dbms/src/Interpreters/Context.h | 7 +- .../Interpreters/InterpreterSelectQuery.cpp | 30 +- dbms/src/Interpreters/Users.cpp | 29 +- dbms/src/Interpreters/Users.h | 9 +- 15 files changed, 783 insertions(+), 83 deletions(-) create mode 100644 dbms/src/Access/RowPolicy.cpp create mode 100644 dbms/src/Access/RowPolicy.h create mode 100644 dbms/src/Access/RowPolicyContext.cpp create mode 100644 dbms/src/Access/RowPolicyContext.h create mode 100644 dbms/src/Access/RowPolicyContextFactory.cpp create mode 100644 dbms/src/Access/RowPolicyContextFactory.h diff --git a/dbms/src/Access/AccessControlManager.cpp b/dbms/src/Access/AccessControlManager.cpp index 1f1a57816a8..249dc54fb09 100644 --- a/dbms/src/Access/AccessControlManager.cpp +++ b/dbms/src/Access/AccessControlManager.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -21,7 +22,8 @@ namespace AccessControlManager::AccessControlManager() : MultipleAccessStorage(createStorages()), - quota_context_factory(std::make_unique(*this)) + quota_context_factory(std::make_unique(*this)), + row_policy_context_factory(std::make_unique(*this)) { } @@ -49,4 +51,11 @@ std::vector AccessControlManager::getQuotaUsageInfo() const { return quota_context_factory->getUsageInfo(); } + + +std::shared_ptr AccessControlManager::getRowPolicyContext(const String & user_name) const +{ + return row_policy_context_factory->createContext(user_name); +} + } diff --git a/dbms/src/Access/AccessControlManager.h b/dbms/src/Access/AccessControlManager.h index 2133717d676..9658dc7161d 100644 --- a/dbms/src/Access/AccessControlManager.h +++ b/dbms/src/Access/AccessControlManager.h @@ -22,6 +22,8 @@ namespace DB class QuotaContext; class QuotaContextFactory; struct QuotaUsageInfo; +class RowPolicyContext; +class RowPolicyContextFactory; /// Manages access control entities. @@ -38,8 +40,11 @@ public: std::vector getQuotaUsageInfo() const; + std::shared_ptr getRowPolicyContext(const String & user_name) const; + private: std::unique_ptr quota_context_factory; + std::unique_ptr row_policy_context_factory; }; } diff --git a/dbms/src/Access/IAccessEntity.cpp b/dbms/src/Access/IAccessEntity.cpp index 6a2f928ae9e..9b203a74148 100644 --- a/dbms/src/Access/IAccessEntity.cpp +++ b/dbms/src/Access/IAccessEntity.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -9,6 +10,8 @@ String IAccessEntity::getTypeName(std::type_index type) { if (type == typeid(Quota)) return "Quota"; + if (type == typeid(RowPolicy)) + return "Row policy"; return demangle(type.name()); } diff --git a/dbms/src/Access/RowPolicy.cpp b/dbms/src/Access/RowPolicy.cpp new file mode 100644 index 00000000000..391303e46a2 --- /dev/null +++ b/dbms/src/Access/RowPolicy.cpp @@ -0,0 +1,111 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + void generateFullNameImpl(const String & database_, const String & table_name_, const String & policy_name_, String & full_name_) + { + full_name_.clear(); + full_name_.reserve(database_.length() + table_name_.length() + policy_name_.length() + 6); + full_name_ += backQuoteIfNeed(policy_name_); + full_name_ += " ON "; + if (!database_.empty()) + { + full_name_ += backQuoteIfNeed(database_); + full_name_ += '.'; + } + full_name_ += backQuoteIfNeed(table_name_); + } +} + + +String RowPolicy::FullNameParts::getFullName() const +{ + String full_name; + generateFullNameImpl(database, table_name, policy_name, full_name); + return full_name; +} + + +String RowPolicy::FullNameParts::getFullName(const Context & context) const +{ + String full_name; + generateFullNameImpl(database.empty() ? context.getCurrentDatabase() : database, table_name, policy_name, full_name); + return full_name; +} + + +void RowPolicy::setDatabase(const String & database_) +{ + database = database_; + generateFullNameImpl(database, table_name, policy_name, full_name); +} + + +void RowPolicy::setTableName(const String & table_name_) +{ + table_name = table_name_; + generateFullNameImpl(database, table_name, policy_name, full_name); +} + + +void RowPolicy::setName(const String & policy_name_) +{ + policy_name = policy_name_; + generateFullNameImpl(database, table_name, policy_name, full_name); +} + + +void RowPolicy::setFullName(const String & database_, const String & table_name_, const String & policy_name_) +{ + database = database_; + table_name = table_name_; + policy_name = policy_name_; + generateFullNameImpl(database, table_name, policy_name, full_name); +} + + +bool RowPolicy::equal(const IAccessEntity & other) const +{ + if (!IAccessEntity::equal(other)) + return false; + const auto & other_policy = typeid_cast(other); + return (database == other_policy.database) && (table_name == other_policy.table_name) && (policy_name == other_policy.policy_name) + && boost::range::equal(conditions, other_policy.conditions) && restrictive == other_policy.restrictive + && (roles == other_policy.roles) && (all_roles == other_policy.all_roles) && (except_roles == other_policy.except_roles); +} + + +const char * RowPolicy::conditionIndexToString(ConditionIndex index) +{ + switch (index) + { + case SELECT_FILTER: return "SELECT_FILTER"; + case INSERT_CHECK: return "INSERT_CHECK"; + case UPDATE_FILTER: return "UPDATE_FILTER"; + case UPDATE_CHECK: return "UPDATE_CHECK"; + case DELETE_FILTER: return "DELETE_FILTER"; + } + __builtin_unreachable(); +} + + +const char * RowPolicy::conditionIndexToColumnName(ConditionIndex index) +{ + switch (index) + { + case SELECT_FILTER: return "select_filter"; + case INSERT_CHECK: return "insert_check"; + case UPDATE_FILTER: return "update_filter"; + case UPDATE_CHECK: return "update_check"; + case DELETE_FILTER: return "delete_filter"; + } + __builtin_unreachable(); +} + +} diff --git a/dbms/src/Access/RowPolicy.h b/dbms/src/Access/RowPolicy.h new file mode 100644 index 00000000000..22681b8875e --- /dev/null +++ b/dbms/src/Access/RowPolicy.h @@ -0,0 +1,81 @@ +#pragma once + +#include + + +namespace DB +{ +class Context; + + +/** Represents a row level security policy for a table. + */ +struct RowPolicy : public IAccessEntity +{ + void setDatabase(const String & database_); + void setTableName(const String & table_name_); + void setName(const String & policy_name_) override; + void setFullName(const String & database_, const String & table_name_, const String & policy_name_); + + String getDatabase() const { return database; } + String getTableName() const { return table_name; } + String getName() const override { return policy_name; } + + struct FullNameParts + { + String database; + String table_name; + String policy_name; + String getFullName() const; + String getFullName(const Context & context) const; + }; + + /// Filter is a SQL conditional expression used to figure out which rows should be visible + /// for user or available for modification. If the expression returns NULL or false for some rows + /// those rows are silently suppressed. + /// Check is a SQL condition expression used to check whether a row can be written into + /// the table. If the expression returns NULL or false an exception is thrown. + /// If a conditional expression here is empty it means no filtering is applied. + enum ConditionIndex + { + SELECT_FILTER, + INSERT_CHECK, + UPDATE_FILTER, + UPDATE_CHECK, + DELETE_FILTER, + }; + static constexpr size_t MAX_CONDITION_INDEX = 5; + static const char * conditionIndexToString(ConditionIndex index); + static const char * conditionIndexToColumnName(ConditionIndex index); + + String conditions[MAX_CONDITION_INDEX]; + + /// Sets that the policy is permissive. + /// A row is only accessible if at least one of the permissive policies passes, + /// in addition to all the restrictive policies. + void setPermissive(bool permissive_ = true) { setRestrictive(!permissive_); } + bool isPermissive() const { return !isRestrictive(); } + + /// Sets that the policy is restrictive. + /// A row is only accessible if at least one of the permissive policies passes, + /// in addition to all the restrictive policies. + void setRestrictive(bool restrictive_ = true) { restrictive = restrictive_; } + bool isRestrictive() const { return restrictive; } + + bool equal(const IAccessEntity & other) const override; + std::shared_ptr clone() const override { return cloneImpl(); } + + /// Which roles or users should use this quota. + Strings roles; + bool all_roles = false; + Strings except_roles; + +private: + String database; + String table_name; + String policy_name; + bool restrictive = false; +}; + +using RowPolicyPtr = std::shared_ptr; +} diff --git a/dbms/src/Access/RowPolicyContext.cpp b/dbms/src/Access/RowPolicyContext.cpp new file mode 100644 index 00000000000..cb24d0af01b --- /dev/null +++ b/dbms/src/Access/RowPolicyContext.cpp @@ -0,0 +1,59 @@ +#include +#include +#include + + +namespace DB +{ +size_t RowPolicyContext::Hash::operator()(const DatabaseAndTableNameRef & database_and_table_name) const +{ + return std::hash{}(database_and_table_name.first) - std::hash{}(database_and_table_name.second); +} + + +RowPolicyContext::RowPolicyContext() + : atomic_map_of_mixed_conditions(std::make_shared()) +{ +} + + +RowPolicyContext::~RowPolicyContext() = default; + + +RowPolicyContext::RowPolicyContext(const String & user_name_) + : user_name(user_name_) +{} + + +ASTPtr RowPolicyContext::getCondition(const String & database, const String & table_name, ConditionIndex index) const +{ + /// We don't lock `mutex` here. + auto map_of_mixed_conditions = std::atomic_load(&atomic_map_of_mixed_conditions); + auto it = map_of_mixed_conditions->find({database, table_name}); + if (it == map_of_mixed_conditions->end()) + return {}; + return it->second.mixed_conditions[index]; +} + + +std::vector RowPolicyContext::getCurrentPolicyIDs() const +{ + /// We don't lock `mutex` here. + auto map_of_mixed_conditions = std::atomic_load(&atomic_map_of_mixed_conditions); + std::vector policy_ids; + for (const auto & mixed_conditions : *map_of_mixed_conditions | boost::adaptors::map_values) + boost::range::copy(mixed_conditions.policy_ids, std::back_inserter(policy_ids)); + return policy_ids; +} + + +std::vector RowPolicyContext::getCurrentPolicyIDs(const String & database, const String & table_name) const +{ + /// We don't lock `mutex` here. + auto map_of_mixed_conditions = std::atomic_load(&atomic_map_of_mixed_conditions); + auto it = map_of_mixed_conditions->find({database, table_name}); + if (it == map_of_mixed_conditions->end()) + return {}; + return it->second.policy_ids; +} +} diff --git a/dbms/src/Access/RowPolicyContext.h b/dbms/src/Access/RowPolicyContext.h new file mode 100644 index 00000000000..776808f74d7 --- /dev/null +++ b/dbms/src/Access/RowPolicyContext.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class IAST; +using ASTPtr = std::shared_ptr; + + +/// Provides fast access to row policies' conditions for a specific user and tables. +class RowPolicyContext +{ +public: + /// Default constructor makes a row policy usage context which restricts nothing. + RowPolicyContext(); + + ~RowPolicyContext(); + + using ConditionIndex = RowPolicy::ConditionIndex; + + /// Returns prepared filter for a specific table and operations. + /// The function can return nullptr, that means there is no filters applied. + /// The returned filter can be a combination of the filters defined by multiple row policies. + ASTPtr getCondition(const String & database, const String & table_name, ConditionIndex index) const; + + /// Returns IDs of all the policies used by the current user. + std::vector getCurrentPolicyIDs() const; + + /// Returns IDs of the policies used by a concrete table. + std::vector getCurrentPolicyIDs(const String & database, const String & table_name) const; + +private: + friend class RowPolicyContextFactory; + friend struct ext::shared_ptr_helper; + RowPolicyContext(const String & user_name_); /// RowPolicyContext should be created by RowPolicyContextFactory. + + using DatabaseAndTableName = std::pair; + using DatabaseAndTableNameRef = std::pair; + struct Hash + { + size_t operator()(const DatabaseAndTableNameRef & database_and_table_name) const; + }; + static constexpr size_t MAX_CONDITION_INDEX = RowPolicy::MAX_CONDITION_INDEX; + using ParsedConditions = std::array; + struct MixedConditions + { + std::unique_ptr database_and_table_name_keeper; + ParsedConditions mixed_conditions; + std::vector policy_ids; + }; + using MapOfMixedConditions = std::unordered_map; + + const String user_name; + std::shared_ptr atomic_map_of_mixed_conditions; /// Changed atomically, not protected by `mutex`. +}; + + +using RowPolicyContextPtr = std::shared_ptr; +} diff --git a/dbms/src/Access/RowPolicyContextFactory.cpp b/dbms/src/Access/RowPolicyContextFactory.cpp new file mode 100644 index 00000000000..e458f06ca94 --- /dev/null +++ b/dbms/src/Access/RowPolicyContextFactory.cpp @@ -0,0 +1,314 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool tryGetLiteralBool(const IAST & ast, bool & value) + { + try + { + if (const ASTLiteral * literal = ast.as()) + { + value = !literal->value.isNull() && applyVisitor(FieldVisitorConvertToNumber(), literal->value); + return true; + } + return false; + } + catch (...) + { + return false; + } + } + + ASTPtr applyFunctionAND(ASTs arguments) + { + bool const_arguments = true; + boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool + { + bool b; + if (!tryGetLiteralBool(*argument, b)) + return false; + const_arguments &= b; + return true; + }); + + if (!const_arguments) + return std::make_shared(Field{UInt8(0)}); + if (arguments.empty()) + return std::make_shared(Field{UInt8(1)}); + if (arguments.size() == 1) + return arguments[0]; + + auto function = std::make_shared(); + auto exp_list = std::make_shared(); + function->name = "and"; + function->arguments = exp_list; + function->children.push_back(exp_list); + exp_list->children = std::move(arguments); + return function; + } + + + ASTPtr applyFunctionOR(ASTs arguments) + { + bool const_arguments = false; + boost::range::remove_erase_if(arguments, [&](const ASTPtr & argument) -> bool + { + bool b; + if (!tryGetLiteralBool(*argument, b)) + return false; + const_arguments |= b; + return true; + }); + + if (const_arguments) + return std::make_shared(Field{UInt8(1)}); + if (arguments.empty()) + return std::make_shared(Field{UInt8(0)}); + if (arguments.size() == 1) + return arguments[0]; + + auto function = std::make_shared(); + auto exp_list = std::make_shared(); + function->name = "or"; + function->arguments = exp_list; + function->children.push_back(exp_list); + exp_list->children = std::move(arguments); + return function; + } + + + using ConditionIndex = RowPolicy::ConditionIndex; + static constexpr size_t MAX_CONDITION_INDEX = RowPolicy::MAX_CONDITION_INDEX; + + + /// Accumulates conditions from multiple row policies and joins them using the AND logical operation. + class ConditionsMixer + { + public: + void add(const ASTPtr & condition, bool is_restrictive) + { + if (!condition) + return; + + if (is_restrictive) + restrictions.push_back(condition); + else + permissions.push_back(condition); + } + + ASTPtr getResult() && + { + /// Process permissive conditions. + if (!permissions.empty()) + restrictions.push_back(applyFunctionOR(std::move(permissions))); + + /// Process restrictive conditions. + if (!restrictions.empty()) + return applyFunctionAND(std::move(restrictions)); + return nullptr; + } + + private: + ASTs permissions; + ASTs restrictions; + }; +} + + +void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) +{ + policy = policy_; + + boost::range::copy(policy->roles, std::inserter(roles, roles.end())); + all_roles = policy->all_roles; + boost::range::copy(policy->except_roles, std::inserter(except_roles, except_roles.end())); + + for (auto index : ext::range_with_static_cast(0, MAX_CONDITION_INDEX)) + { + const String & condition = policy->conditions[index]; + auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + index); + auto previous_it = std::find(previous_range.first, previous_range.second, condition); + if (previous_it != previous_range.second) + { + /// The condition is already parsed before. + parsed_conditions[index] = parsed_conditions[previous_it - previous_range.first]; + } + else + { + /// Try to parse the condition. + try + { + ParserExpression parser; + parsed_conditions[index] = parseQuery(parser, condition, 0); + } + catch (...) + { + tryLogCurrentException( + &Poco::Logger::get("RowPolicy"), + String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy " + + backQuote(policy->getFullName())); + } + } + } +} + + +bool RowPolicyContextFactory::PolicyInfo::canUseWithContext(const RowPolicyContext & context) const +{ + if (roles.count(context.user_name)) + return true; + + if (all_roles && !except_roles.count(context.user_name)) + return true; + + return false; +} + + +RowPolicyContextFactory::RowPolicyContextFactory(const AccessControlManager & access_control_manager_) + : access_control_manager(access_control_manager_) +{ +} + +RowPolicyContextFactory::~RowPolicyContextFactory() = default; + + +RowPolicyContextPtr RowPolicyContextFactory::createContext(const String & user_name) +{ + std::lock_guard lock{mutex}; + ensureAllRowPoliciesRead(); + auto context = ext::shared_ptr_helper::create(user_name); + contexts.push_back(context); + mixConditionsForContext(*context); + return context; +} + + +void RowPolicyContextFactory::ensureAllRowPoliciesRead() +{ + /// `mutex` is already locked. + if (all_policies_read) + return; + all_policies_read = true; + + subscription = access_control_manager.subscribeForChanges( + [&](const UUID & id, const AccessEntityPtr & entity) + { + if (entity) + rowPolicyAddedOrChanged(id, typeid_cast(entity)); + else + rowPolicyRemoved(id); + }); + + for (const UUID & id : access_control_manager.findAll()) + { + auto quota = access_control_manager.tryRead(id); + if (quota) + all_policies.emplace(id, PolicyInfo(quota)); + } +} + + +void RowPolicyContextFactory::rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy) +{ + std::lock_guard lock{mutex}; + auto it = all_policies.find(policy_id); + if (it == all_policies.end()) + { + it = all_policies.emplace(policy_id, PolicyInfo(new_policy)).first; + } + else + { + if (it->second.policy == new_policy) + return; + } + + auto & info = it->second; + info.setPolicy(new_policy); + mixConditionsForAllContexts(); +} + + +void RowPolicyContextFactory::rowPolicyRemoved(const UUID & policy_id) +{ + std::lock_guard lock{mutex}; + all_policies.erase(policy_id); + mixConditionsForAllContexts(); +} + + +void RowPolicyContextFactory::mixConditionsForAllContexts() +{ + /// `mutex` is already locked. + boost::range::remove_erase_if( + contexts, + [&](const std::weak_ptr & weak) + { + auto context = weak.lock(); + if (!context) + return true; // remove from the `contexts` list. + mixConditionsForContext(*context); + return false; // keep in the `contexts` list. + }); +} + + +void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context) +{ + /// `mutex` is already locked. + struct Mixers + { + ConditionsMixer mixers[MAX_CONDITION_INDEX]; + std::vector policy_ids; + }; + using MapOfMixedConditions = RowPolicyContext::MapOfMixedConditions; + using DatabaseAndTableName = RowPolicyContext::DatabaseAndTableName; + using DatabaseAndTableNameRef = RowPolicyContext::DatabaseAndTableNameRef; + using Hash = RowPolicyContext::Hash; + + std::unordered_map map_of_mixers; + + for (const auto & [policy_id, info] : all_policies) + { + if (info.canUseWithContext(context)) + { + const auto & policy = *info.policy; + auto & mixers = map_of_mixers[std::pair{policy.getDatabase(), policy.getTableName()}]; + mixers.policy_ids.push_back(policy_id); + for (auto index : ext::range(0, MAX_CONDITION_INDEX)) + mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive()); + } + } + + auto map_of_mixed_conditions = std::make_shared(); + for (auto & [database_and_table_name, mixers] : map_of_mixers) + { + auto database_and_table_name_keeper = std::make_unique(); + database_and_table_name_keeper->first = database_and_table_name.first; + database_and_table_name_keeper->second = database_and_table_name.second; + auto & mixed_conditions = (*map_of_mixed_conditions)[DatabaseAndTableNameRef{database_and_table_name_keeper->first, + database_and_table_name_keeper->second}]; + mixed_conditions.database_and_table_name_keeper = std::move(database_and_table_name_keeper); + mixed_conditions.policy_ids = std::move(mixers.policy_ids); + for (auto index : ext::range(0, MAX_CONDITION_INDEX)) + mixed_conditions.mixed_conditions[index] = std::move(mixers.mixers[index]).getResult(); + } + + std::atomic_store(&context.atomic_map_of_mixed_conditions, std::shared_ptr{map_of_mixed_conditions}); +} + +} diff --git a/dbms/src/Access/RowPolicyContextFactory.h b/dbms/src/Access/RowPolicyContextFactory.h new file mode 100644 index 00000000000..02019c468a9 --- /dev/null +++ b/dbms/src/Access/RowPolicyContextFactory.h @@ -0,0 +1,54 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ +class AccessControlManager; + + +/// Stores read and parsed row policies. +class RowPolicyContextFactory +{ +public: + RowPolicyContextFactory(const AccessControlManager & access_control_manager_); + ~RowPolicyContextFactory(); + + RowPolicyContextPtr createContext(const String & user_name); + +private: + using ParsedConditions = RowPolicyContext::ParsedConditions; + + struct PolicyInfo + { + PolicyInfo(const RowPolicyPtr & policy_) { setPolicy(policy_); } + void setPolicy(const RowPolicyPtr & policy_); + bool canUseWithContext(const RowPolicyContext & context) const; + + RowPolicyPtr policy; + std::unordered_set roles; + bool all_roles = false; + std::unordered_set except_roles; + ParsedConditions parsed_conditions; + }; + + void ensureAllRowPoliciesRead(); + void rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy); + void rowPolicyRemoved(const UUID & policy_id); + void mixConditionsForAllContexts(); + void mixConditionsForContext(RowPolicyContext & context); + + const AccessControlManager & access_control_manager; + std::unordered_map all_policies; + bool all_policies_read = false; + IAccessStorage::SubscriptionPtr subscription; + std::vector> contexts; + std::mutex mutex; +}; + +} diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp index d417968bb64..c9671afaca1 100644 --- a/dbms/src/Access/UsersConfigAccessStorage.cpp +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -15,6 +16,8 @@ namespace { if (type == typeid(Quota)) return 'Q'; + if (type == typeid(RowPolicy)) + return 'P'; return 0; } @@ -112,6 +115,57 @@ namespace } return quotas; } + + + std::vector parseRowPolicies(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) + { + std::vector policies; + Poco::Util::AbstractConfiguration::Keys user_names; + config.keys("users", user_names); + + for (const String & user_name : user_names) + { + const String databases_config = "users." + user_name + ".databases"; + if (config.has(databases_config)) + { + Poco::Util::AbstractConfiguration::Keys databases; + config.keys(databases_config, databases); + + /// Read tables within databases + for (const String & database : databases) + { + const String database_config = databases_config + "." + database; + Poco::Util::AbstractConfiguration::Keys table_names; + config.keys(database_config, table_names); + + /// Read table properties + for (const String & table_name : table_names) + { + const auto filter_config = database_config + "." + table_name + ".filter"; + if (config.has(filter_config)) + { + try + { + auto policy = std::make_shared(); + policy->setFullName(database, table_name, user_name); + policy->conditions[RowPolicy::SELECT_FILTER] = config.getString(filter_config); + policy->roles.push_back(user_name); + policies.push_back(policy); + } + catch (...) + { + tryLogCurrentException( + log, + "Could not parse row policy " + backQuote(user_name) + " on table " + backQuoteIfNeed(database) + "." + + backQuoteIfNeed(table_name)); + } + } + } + } + } + } + return policies; + } } @@ -128,6 +182,8 @@ void UsersConfigAccessStorage::loadFromConfig(const Poco::Util::AbstractConfigur std::vector> all_entities; for (const auto & entity : parseQuotas(config, getLogger())) all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseRowPolicies(config, getLogger())) + all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 4ed8a540f29..e6b54f3dbc4 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -333,6 +334,7 @@ Context Context::createGlobal() { Context res; res.quota = std::make_shared(); + res.row_policy = std::make_shared(); res.shared = std::make_shared(); return res; } @@ -639,34 +641,6 @@ ConfigurationPtr Context::getUsersConfig() 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() { auto lock = getLock(); @@ -691,6 +665,7 @@ void Context::calculateUserSettings() quota = getAccessControlManager().createQuotaContext( client_info.current_user, client_info.current_address.host(), client_info.quota_key); is_quota_management_allowed = user->is_quota_management_allowed; + row_policy = getAccessControlManager().getRowPolicyContext(client_info.current_user); } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 930b0f254fe..f434df05238 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -45,6 +45,7 @@ namespace DB struct ContextShared; class Context; class QuotaContext; +class RowPolicyContext; class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalModelsLoader; @@ -140,6 +141,7 @@ private: std::shared_ptr quota; /// Current quota. By default - empty quota, that have no limits. bool is_quota_management_allowed = false; /// Whether the current user is allowed to manage quotas via SQL commands. + std::shared_ptr row_policy; String current_database; Settings settings; /// Setting for query execution. std::shared_ptr settings_constraints; @@ -209,6 +211,7 @@ public: const AccessControlManager & getAccessControlManager() const; std::shared_ptr getQuota() const { return quota; } void checkQuotaManagementIsAllowed(); + std::shared_ptr getRowPolicy() const { return row_policy; } /** Take the list of users, quotas and configuration profiles from this config. * The list of users is completely replaced. @@ -217,10 +220,6 @@ public: void setUsersConfig(const ConfigurationPtr & config); ConfigurationPtr getUsersConfig(); - // User property is a key-value pair from the configuration entry: users..databases... - 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. void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0ba5c9ecd6c..389309c91b5 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -38,6 +38,8 @@ #include #include +#include + #include #include #include @@ -118,11 +120,10 @@ namespace { /// Assumes `storage` is set and the table filter (row-level security) is not empty. -String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr & storage, const Context & context, const Names & prerequisite_columns = {}) +String generateFilterActions(ExpressionActionsPtr & actions, const Context & context, const StoragePtr & storage, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) { const auto & db_name = storage->getDatabaseName(); const auto & table_name = storage->getTableName(); - const auto & filter_str = context.getUserProperty(db_name, table_name, "filter"); /// TODO: implement some AST builders for this kind of stuff ASTPtr query_ast = std::make_shared(); @@ -131,18 +132,15 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr & select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); auto expr_list = select_ast->select(); - auto parseExpression = [] (const String & expr) - { - ParserExpression expr_parser; - return parseQuery(expr_parser, expr, 0); - }; - // The first column is our filter expression. - expr_list->children.push_back(parseExpression(filter_str)); + expr_list->children.push_back(row_policy_filter); /// Keep columns that are required after the filter actions. for (const auto & column_str : prerequisite_columns) - expr_list->children.push_back(parseExpression(column_str)); + { + ParserExpression expr_parser; + expr_list->children.push_back(parseQuery(expr_parser, column_str, 0)); + } select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); auto tables = select_ast->tables(); @@ -378,10 +376,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = storage->getSampleBlockForColumns(required_columns); /// Fix source_header for filter actions. - if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + auto row_policy_filter = context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER); + if (row_policy_filter) { filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions, storage, *context, required_columns); + filter_info->column_name = generateFilterActions(filter_info->actions, *context, storage, row_policy_filter, required_columns); source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns()); } } @@ -502,7 +501,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() /// PREWHERE optimization. /// Turn off, if the table filter (row-level security) is applied. - if (storage && !context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (storage && !context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER)) { query_analyzer->makeSetsForIndex(query.where()); query_analyzer->makeSetsForIndex(query.prewhere()); @@ -1443,11 +1442,12 @@ void InterpreterSelectQuery::executeFetchColumns( if (storage) { /// Append columns from the table filter to required - if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + auto row_policy_filter = context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER); + if (row_policy_filter) { auto initial_required_columns = required_columns; ExpressionActionsPtr actions; - generateFilterActions(actions, storage, *context, initial_required_columns); + generateFilterActions(actions, *context, storage, row_policy_filter, initial_required_columns); auto required_columns_from_filter = actions->getRequiredColumns(); for (const auto & column : required_columns_from_filter) diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index e66b5119f84..81898ba5135 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -102,34 +103,6 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A } } - /// 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; - } - } - } - } - if (config.has(config_elem + ".allow_quota_management")) is_quota_management_allowed = config.getBool(config_elem + ".allow_quota_management"); } diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index 6f9a47c4422..189bfb991d5 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -1,12 +1,13 @@ #pragma once #include +#include #include #include #include -#include #include +#include namespace Poco @@ -41,12 +42,6 @@ struct User using DictionarySet = std::unordered_set; std::optional dictionaries; - /// Table properties. - using PropertyMap = std::unordered_map; - using TableMap = std::unordered_map; - using DatabaseMap = std::unordered_map; - DatabaseMap table_props; - bool is_quota_management_allowed = false; User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config); From 754fb40cc4815524d5136477ac0ef10779ca8a5c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Nov 2019 22:44:44 +0300 Subject: [PATCH 07/66] Transform stateless test for row policy into an integration test with improvements. --- dbms/programs/server/users.xml | 14 +- .../integration/test_row_policy/__init__.py | 0 .../integration/test_row_policy/all_rows.xml | 23 +++ .../configs/users.d/row_policy.xml | 1 + .../test_row_policy/configs/users.xml | 23 +++ .../test_row_policy/no_filters.xml | 3 + .../integration/test_row_policy/no_rows.xml | 23 +++ .../test_row_policy/normal_filters.xml | 26 ++++ .../tests/integration/test_row_policy/test.py | 139 ++++++++++++++++++ .../0_stateless/00927_table_filter.reference | 32 ---- .../0_stateless/00927_table_filter.sql | 46 ------ 11 files changed, 241 insertions(+), 89 deletions(-) create mode 100644 dbms/tests/integration/test_row_policy/__init__.py create mode 100644 dbms/tests/integration/test_row_policy/all_rows.xml create mode 120000 dbms/tests/integration/test_row_policy/configs/users.d/row_policy.xml create mode 100644 dbms/tests/integration/test_row_policy/configs/users.xml create mode 100644 dbms/tests/integration/test_row_policy/no_filters.xml create mode 100644 dbms/tests/integration/test_row_policy/no_rows.xml create mode 100644 dbms/tests/integration/test_row_policy/normal_filters.xml create mode 100644 dbms/tests/integration/test_row_policy/test.py delete mode 100644 dbms/tests/queries/0_stateless/00927_table_filter.reference delete mode 100644 dbms/tests/queries/0_stateless/00927_table_filter.sql diff --git a/dbms/programs/server/users.xml b/dbms/programs/server/users.xml index 9755c29d480..0058ee51184 100644 --- a/dbms/programs/server/users.xml +++ b/dbms/programs/server/users.xml @@ -84,25 +84,17 @@ default - - + + a = 1 - - a + b < 1 or c - d > 5 - - - - c = 1 - - + --> diff --git a/dbms/tests/integration/test_row_policy/__init__.py b/dbms/tests/integration/test_row_policy/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_row_policy/all_rows.xml b/dbms/tests/integration/test_row_policy/all_rows.xml new file mode 100644 index 00000000000..d00db61482c --- /dev/null +++ b/dbms/tests/integration/test_row_policy/all_rows.xml @@ -0,0 +1,23 @@ + + + + + + + + + 1 + + + + 1 + + + + 1 + + + + + + diff --git a/dbms/tests/integration/test_row_policy/configs/users.d/row_policy.xml b/dbms/tests/integration/test_row_policy/configs/users.d/row_policy.xml new file mode 120000 index 00000000000..593f0e42662 --- /dev/null +++ b/dbms/tests/integration/test_row_policy/configs/users.d/row_policy.xml @@ -0,0 +1 @@ +../../normal_filters.xml \ No newline at end of file diff --git a/dbms/tests/integration/test_row_policy/configs/users.xml b/dbms/tests/integration/test_row_policy/configs/users.xml new file mode 100644 index 00000000000..c86ab1f15a1 --- /dev/null +++ b/dbms/tests/integration/test_row_policy/configs/users.xml @@ -0,0 +1,23 @@ + + + + + 1 + + + + + + + ::/0 + + default + default + true + + + + + + + diff --git a/dbms/tests/integration/test_row_policy/no_filters.xml b/dbms/tests/integration/test_row_policy/no_filters.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_row_policy/no_filters.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_row_policy/no_rows.xml b/dbms/tests/integration/test_row_policy/no_rows.xml new file mode 100644 index 00000000000..904d94b13fb --- /dev/null +++ b/dbms/tests/integration/test_row_policy/no_rows.xml @@ -0,0 +1,23 @@ + + + + + + + + + NULL + + + + NULL + + + + NULL + + + + + + diff --git a/dbms/tests/integration/test_row_policy/normal_filters.xml b/dbms/tests/integration/test_row_policy/normal_filters.xml new file mode 100644 index 00000000000..ddd84e43ac7 --- /dev/null +++ b/dbms/tests/integration/test_row_policy/normal_filters.xml @@ -0,0 +1,26 @@ + + + + + + + + + + a = 1 + + + + + a + b < 1 or c - d > 5 + + + + + c = 1 + + + + + + diff --git a/dbms/tests/integration/test_row_policy/test.py b/dbms/tests/integration/test_row_policy/test.py new file mode 100644 index 00000000000..bc5061e6704 --- /dev/null +++ b/dbms/tests/integration/test_row_policy/test.py @@ -0,0 +1,139 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +import os +import re +import time + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', + config_dir="configs") + + +def copy_policy_xml(local_file_name, reload_immediately = True): + script_dir = os.path.dirname(os.path.realpath(__file__)) + instance.copy_file_to_container(os.path.join(script_dir, local_file_name), '/etc/clickhouse-server/users.d/row_policy.xml') + if reload_immediately: + instance.query("SYSTEM RELOAD CONFIG") + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + instance.query(''' + CREATE DATABASE mydb; + + CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; + INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1); + + CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a; + INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0); + + CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; + INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1); + ''') + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_policies(): + try: + yield + finally: + copy_policy_xml('normal_filters.xml') + instance.query("DROP POLICY IF EXISTS pA, pB ON mydb.filtered_table1") + + +def test_smoke(): + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t1\n1\t0\n" + + assert instance.query("SELECT a FROM mydb.filtered_table1") == "1\n1\n" + assert instance.query("SELECT b FROM mydb.filtered_table1") == "0\n1\n" + assert instance.query("SELECT a FROM mydb.filtered_table1 WHERE a = 1") == "1\n1\n" + assert instance.query("SELECT a = 1 FROM mydb.filtered_table1") == "1\n1\n" + + assert instance.query("SELECT a FROM mydb.filtered_table3") == "0\n1\n" + assert instance.query("SELECT b FROM mydb.filtered_table3") == "1\n0\n" + assert instance.query("SELECT c FROM mydb.filtered_table3") == "1\n1\n" + assert instance.query("SELECT a + b FROM mydb.filtered_table3") == "1\n1\n" + assert instance.query("SELECT a FROM mydb.filtered_table3 WHERE c = 1") == "0\n1\n" + assert instance.query("SELECT c = 1 FROM mydb.filtered_table3") == "1\n1\n" + assert instance.query("SELECT a + b = 1 FROM mydb.filtered_table3") == "1\n1\n" + + +def test_join(): + assert instance.query("SELECT * FROM mydb.filtered_table1 as t1 ANY LEFT JOIN mydb.filtered_table1 as t2 ON t1.a = t2.b") == "1\t0\t1\t1\n1\t1\t1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table1 as t2 ANY RIGHT JOIN mydb.filtered_table1 as t1 ON t2.b = t1.a") == "1\t1\t1\t0\n" + + +def test_cannot_trick_row_policy_with_keyword_with(): + assert instance.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("WITH 0 AS a SELECT a FROM mydb.filtered_table1") == "1\n1\n" + assert instance.query("WITH 0 AS a SELECT b FROM mydb.filtered_table1") == "0\n1\n" + + +def test_prewhere_not_supported(): + expected_error = "PREWHERE is not supported if the table is filtered by row-level security" + assert expected_error in instance.query_and_get_error("SELECT * FROM mydb.filtered_table1 PREWHERE 1") + assert expected_error in instance.query_and_get_error("SELECT * FROM mydb.filtered_table2 PREWHERE 1") + assert expected_error in instance.query_and_get_error("SELECT * FROM mydb.filtered_table3 PREWHERE 1") + + +def test_change_of_users_xml_changes_row_policies(): + copy_policy_xml('normal_filters.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t1\n1\t0\n" + + copy_policy_xml('all_rows.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "0\t0\n0\t1\n1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n1\t2\t3\t4\n4\t3\t2\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t0\n0\t1\n1\t0\n1\t1\n" + + copy_policy_xml('no_rows.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "" + + copy_policy_xml('normal_filters.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t1\n1\t0\n" + + copy_policy_xml('no_filters.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "0\t0\n0\t1\n1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n1\t2\t3\t4\n4\t3\t2\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t0\n0\t1\n1\t0\n1\t1\n" + + copy_policy_xml('normal_filters.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t1\n1\t0\n" + + +def test_reload_users_xml_by_timer(): + copy_policy_xml('normal_filters.xml') + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n1\t1\n" + assert instance.query("SELECT * FROM mydb.filtered_table2") == "0\t0\t0\t0\n0\t0\t6\t0\n" + assert instance.query("SELECT * FROM mydb.filtered_table3") == "0\t1\n1\t0\n" + + time.sleep(1) # The modification time of the 'row_policy.xml' file should be different. + copy_policy_xml('all_rows.xml', False) + assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table1", "0\t0\n0\t1\n1\t0\n1\t1") + assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table2", "0\t0\t0\t0\n0\t0\t6\t0\n1\t2\t3\t4\n4\t3\t2\t1") + assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table3", "0\t0\n0\t1\n1\t0\n1\t1") + + time.sleep(1) # The modification time of the 'row_policy.xml' file should be different. + copy_policy_xml('normal_filters.xml', False) + assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table1", "1\t0\n1\t1") + assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table2", "0\t0\t0\t0\n0\t0\t6\t0") + assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table3", "0\t1\n1\t0") diff --git a/dbms/tests/queries/0_stateless/00927_table_filter.reference b/dbms/tests/queries/0_stateless/00927_table_filter.reference deleted file mode 100644 index e3ea6e2f0ff..00000000000 --- a/dbms/tests/queries/0_stateless/00927_table_filter.reference +++ /dev/null @@ -1,32 +0,0 @@ --- 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 diff --git a/dbms/tests/queries/0_stateless/00927_table_filter.sql b/dbms/tests/queries/0_stateless/00927_table_filter.sql deleted file mode 100644 index ff9703be6cd..00000000000 --- a/dbms/tests/queries/0_stateless/00927_table_filter.sql +++ /dev/null @@ -1,46 +0,0 @@ -SET any_join_distinct_right_table_keys = 1; - -DROP TABLE IF EXISTS test.filtered_table1; -DROP TABLE IF EXISTS test.filtered_table2; -DROP TABLE IF EXISTS test.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; From 6baccb963d3a69919f0a17ab7f8f4abe88675ab2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 3 Dec 2019 21:19:11 +0300 Subject: [PATCH 08/66] Add functions currentRowPolicies() and system table 'system.row_policies'. --- dbms/src/Functions/currentRowPolicies.cpp | 225 ++++++++++++++++++ dbms/src/Functions/registerFunctions.h | 1 + .../registerFunctionsMiscellaneous.cpp | 1 + .../System/StorageSystemRowPolicies.cpp | 59 +++++ .../System/StorageSystemRowPolicies.h | 26 ++ .../Storages/System/attachSystemTables.cpp | 2 + .../tests/integration/test_row_policy/test.py | 16 ++ 7 files changed, 330 insertions(+) create mode 100644 dbms/src/Functions/currentRowPolicies.cpp create mode 100644 dbms/src/Storages/System/StorageSystemRowPolicies.cpp create mode 100644 dbms/src/Storages/System/StorageSystemRowPolicies.h diff --git a/dbms/src/Functions/currentRowPolicies.cpp b/dbms/src/Functions/currentRowPolicies.cpp new file mode 100644 index 00000000000..4ffd40ed1b9 --- /dev/null +++ b/dbms/src/Functions/currentRowPolicies.cpp @@ -0,0 +1,225 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/// The currentRowPolicies() function can be called with 0..2 arguments: +/// currentRowPolicies() returns array of tuples (database, table_name, row_policy_name) for all the row policies applied for the current user; +/// currentRowPolicies(table_name) is equivalent to currentRowPolicies(currentDatabase(), table_name); +/// currentRowPolicies(database, table_name) returns array of names of the row policies applied to a specific table and for the current user. +class FunctionCurrentRowPolicies : public IFunction +{ +public: + static constexpr auto name = "currentRowPolicies"; + + static FunctionPtr create(const Context & context_) { return std::make_shared(context_); } + explicit FunctionCurrentRowPolicies(const Context & context_) : context(context_) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + + void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override + { + if (number_of_arguments > 2) + throw Exception("Number of arguments for function " + String(name) + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 0..2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + return std::make_shared(std::make_shared( + DataTypes{std::make_shared(), std::make_shared(), std::make_shared()})); + else + return std::make_shared(std::make_shared()); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) override + { + if (arguments.empty()) + { + auto database_column = ColumnString::create(); + auto table_name_column = ColumnString::create(); + auto policy_name_column = ColumnString::create(); + for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs()) + { + const auto policy = context.getAccessControlManager().tryRead(policy_id); + if (policy) + { + const String database = policy->getDatabase(); + const String table_name = policy->getTableName(); + const String policy_name = policy->getName(); + database_column->insertData(database.data(), database.length()); + table_name_column->insertData(table_name.data(), table_name.length()); + policy_name_column->insertData(policy_name.data(), policy_name.length()); + } + } + auto offset_column = ColumnArray::ColumnOffsets::create(); + offset_column->insertValue(policy_name_column->size()); + block.getByPosition(result_pos).column = ColumnConst::create( + ColumnArray::create( + ColumnTuple::create(Columns{std::move(database_column), std::move(table_name_column), std::move(policy_name_column)}), + std::move(offset_column)), + input_rows_count); + return; + } + + const IColumn * database_column = nullptr; + if (arguments.size() == 2) + { + const auto & database_column_with_type = block.getByPosition(arguments[0]); + if (!isStringOrFixedString(database_column_with_type.type)) + throw Exception{"The first argument of function " + String(name) + + " should be a string containing database name, illegal type: " + + database_column_with_type.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + database_column = database_column_with_type.column.get(); + } + + const auto & table_name_column_with_type = block.getByPosition(arguments[arguments.size() - 1]); + if (!isStringOrFixedString(table_name_column_with_type.type)) + throw Exception{"The" + String(database_column ? " last" : "") + " argument of function " + String(name) + + " should be a string containing table name, illegal type: " + table_name_column_with_type.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + const IColumn * table_name_column = table_name_column_with_type.column.get(); + + auto policy_name_column = ColumnString::create(); + auto offset_column = ColumnArray::ColumnOffsets::create(); + for (const auto i : ext::range(0, input_rows_count)) + { + String database = database_column ? database_column->getDataAt(i).toString() : context.getCurrentDatabase(); + String table_name = table_name_column->getDataAt(i).toString(); + for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs(database, table_name)) + { + const auto policy = context.getAccessControlManager().tryRead(policy_id); + if (policy) + { + const String policy_name = policy->getName(); + policy_name_column->insertData(policy_name.data(), policy_name.length()); + } + } + offset_column->insertValue(policy_name_column->size()); + } + + block.getByPosition(result_pos).column = ColumnArray::create(std::move(policy_name_column), std::move(offset_column)); + } + +private: + const Context & context; +}; + + +/// The currentRowPolicyIDs() function can be called with 0..2 arguments: +/// currentRowPolicyIDs() returns array of IDs of all the row policies applied for the current user; +/// currentRowPolicyIDs(table_name) is equivalent to currentRowPolicyIDs(currentDatabase(), table_name); +/// currentRowPolicyIDs(database, table_name) returns array of IDs of the row policies applied to a specific table and for the current user. +class FunctionCurrentRowPolicyIDs : public IFunction +{ +public: + static constexpr auto name = "currentRowPolicyIDs"; + + static FunctionPtr create(const Context & context_) { return std::make_shared(context_); } + explicit FunctionCurrentRowPolicyIDs(const Context & context_) : context(context_) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + + void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override + { + if (number_of_arguments > 2) + throw Exception("Number of arguments for function " + String(name) + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 0..2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /* arguments */) const override + { + return std::make_shared(std::make_shared()); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) override + { + if (arguments.empty()) + { + auto policy_id_column = ColumnVector::create(); + for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs()) + policy_id_column->insertValue(policy_id); + auto offset_column = ColumnArray::ColumnOffsets::create(); + offset_column->insertValue(policy_id_column->size()); + block.getByPosition(result_pos).column + = ColumnConst::create(ColumnArray::create(std::move(policy_id_column), std::move(offset_column)), input_rows_count); + return; + } + + const IColumn * database_column = nullptr; + if (arguments.size() == 2) + { + const auto & database_column_with_type = block.getByPosition(arguments[0]); + if (!isStringOrFixedString(database_column_with_type.type)) + throw Exception{"The first argument of function " + String(name) + + " should be a string containing database name, illegal type: " + + database_column_with_type.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + database_column = database_column_with_type.column.get(); + } + + const auto & table_name_column_with_type = block.getByPosition(arguments[arguments.size() - 1]); + if (!isStringOrFixedString(table_name_column_with_type.type)) + throw Exception{"The" + String(database_column ? " last" : "") + " argument of function " + String(name) + + " should be a string containing table name, illegal type: " + table_name_column_with_type.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + const IColumn * table_name_column = table_name_column_with_type.column.get(); + + auto policy_id_column = ColumnVector::create(); + auto offset_column = ColumnArray::ColumnOffsets::create(); + for (const auto i : ext::range(0, input_rows_count)) + { + String database = database_column ? database_column->getDataAt(i).toString() : context.getCurrentDatabase(); + String table_name = table_name_column->getDataAt(i).toString(); + for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs(database, table_name)) + policy_id_column->insertValue(policy_id); + offset_column->insertValue(policy_id_column->size()); + } + + block.getByPosition(result_pos).column = ColumnArray::create(std::move(policy_id_column), std::move(offset_column)); + } + +private: + const Context & context; +}; + + +void registerFunctionCurrentRowPolicies(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctions.h b/dbms/src/Functions/registerFunctions.h index 087fd6b7e2b..5827ae5894c 100644 --- a/dbms/src/Functions/registerFunctions.h +++ b/dbms/src/Functions/registerFunctions.h @@ -9,6 +9,7 @@ class FunctionFactory; void registerFunctionCurrentDatabase(FunctionFactory &); void registerFunctionCurrentUser(FunctionFactory &); void registerFunctionCurrentQuota(FunctionFactory &); +void registerFunctionCurrentRowPolicies(FunctionFactory &); void registerFunctionHostName(FunctionFactory &); void registerFunctionFQDN(FunctionFactory &); void registerFunctionVisibleWidth(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index c45ccf57f64..98c749189d4 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -8,6 +8,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionCurrentDatabase(factory); registerFunctionCurrentUser(factory); registerFunctionCurrentQuota(factory); + registerFunctionCurrentRowPolicies(factory); registerFunctionHostName(factory); registerFunctionFQDN(factory); registerFunctionVisibleWidth(factory); diff --git a/dbms/src/Storages/System/StorageSystemRowPolicies.cpp b/dbms/src/Storages/System/StorageSystemRowPolicies.cpp new file mode 100644 index 00000000000..8ac4ac1b755 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemRowPolicies.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"name", std::make_shared()}, + {"full_name", std::make_shared()}, + {"id", std::make_shared()}, + {"source", std::make_shared()}, + {"restrictive", std::make_shared()}, + }; + + for (auto index : ext::range_with_static_cast(RowPolicy::MAX_CONDITION_INDEX)) + names_and_types.push_back({RowPolicy::conditionIndexToColumnName(index), std::make_shared()}); + + return names_and_types; +} + + +void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +{ + const auto & access_control = context.getAccessControlManager(); + std::vector ids = access_control.findAll(); + + for (const auto & id : ids) + { + auto policy = access_control.tryRead(id); + if (!policy) + continue; + const auto * storage = access_control.findStorage(id); + + size_t i = 0; + res_columns[i++]->insert(policy->getDatabase()); + res_columns[i++]->insert(policy->getTableName()); + res_columns[i++]->insert(policy->getName()); + res_columns[i++]->insert(policy->getFullName()); + res_columns[i++]->insert(id); + res_columns[i++]->insert(storage ? storage->getStorageName() : ""); + res_columns[i++]->insert(policy->isRestrictive()); + + for (auto index : ext::range(RowPolicy::MAX_CONDITION_INDEX)) + res_columns[i++]->insert(policy->conditions[index]); + } +} +} diff --git a/dbms/src/Storages/System/StorageSystemRowPolicies.h b/dbms/src/Storages/System/StorageSystemRowPolicies.h new file mode 100644 index 00000000000..c28342eb18c --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemRowPolicies.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/// Implements `row_policies` system table, which allows you to get information about row policies. +class StorageSystemRowPolicies : public ext::shared_ptr_helper, public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemRowPolicies"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + friend struct ext::shared_ptr_helper; + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override; +}; + +} diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 2b8e630cbed..e8e265ca1e8 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -56,6 +57,7 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("settings", StorageSystemSettings::create("settings")); system_database.attachTable("quotas", StorageSystemQuotas::create("quotas")); system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage")); + system_database.attachTable("row_policies", StorageSystemRowPolicies::create("row_policies")); system_database.attachTable("merge_tree_settings", SystemMergeTreeSettings::create("merge_tree_settings")); system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options")); system_database.attachTable("formats", StorageSystemFormats::create("formats")); diff --git a/dbms/tests/integration/test_row_policy/test.py b/dbms/tests/integration/test_row_policy/test.py index bc5061e6704..3de63d56fa0 100644 --- a/dbms/tests/integration/test_row_policy/test.py +++ b/dbms/tests/integration/test_row_policy/test.py @@ -137,3 +137,19 @@ def test_reload_users_xml_by_timer(): assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table1", "1\t0\n1\t1") assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table2", "0\t0\t0\t0\n0\t0\t6\t0") assert_eq_with_retry(instance, "SELECT * FROM mydb.filtered_table3", "0\t1\n1\t0") + + +def test_introspection(): + assert instance.query("SELECT currentRowPolicies('mydb', 'filtered_table1')") == "['default']\n" + assert instance.query("SELECT currentRowPolicies('mydb', 'filtered_table2')") == "['default']\n" + assert instance.query("SELECT currentRowPolicies('mydb', 'filtered_table3')") == "['default']\n" + assert instance.query("SELECT arraySort(currentRowPolicies())") == "[('mydb','filtered_table1','default'),('mydb','filtered_table2','default'),('mydb','filtered_table3','default')]\n" + + policy1 = "mydb\tfiltered_table1\tdefault\tdefault ON mydb.filtered_table1\t9e8a8f62-4965-2b5e-8599-57c7b99b3549\tusers.xml\t0\ta = 1\t\t\t\t\n" + policy2 = "mydb\tfiltered_table2\tdefault\tdefault ON mydb.filtered_table2\tcffae79d-b9bf-a2ef-b798-019c18470b25\tusers.xml\t0\ta + b < 1 or c - d > 5\t\t\t\t\n" + policy3 = "mydb\tfiltered_table3\tdefault\tdefault ON mydb.filtered_table3\t12fc5cef-e3da-3940-ec79-d8be3911f42b\tusers.xml\t0\tc = 1\t\t\t\t\n" + assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs('mydb', 'filtered_table1'), id) ORDER BY table, name") == policy1 + assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs('mydb', 'filtered_table2'), id) ORDER BY table, name") == policy2 + assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs('mydb', 'filtered_table3'), id) ORDER BY table, name") == policy3 + assert instance.query("SELECT * from system.row_policies ORDER BY table, name") == policy1 + policy2 + policy3 + assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs(), id) ORDER BY table, name") == policy1 + policy2 + policy3 From c3161b7610775f3f3edf134e995e0980a72f8fe3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 29 Nov 2019 20:22:56 +0300 Subject: [PATCH 09/66] Add DCL to manage row policies. --- dbms/src/Interpreters/Context.cpp | 8 + dbms/src/Interpreters/Context.h | 2 + .../InterpreterCreateRowPolicyQuery.cpp | 93 +++++++ .../InterpreterCreateRowPolicyQuery.h | 26 ++ .../InterpreterDropAccessEntityQuery.cpp | 15 + dbms/src/Interpreters/InterpreterFactory.cpp | 12 + ...InterpreterShowCreateAccessEntityQuery.cpp | 51 +++- .../InterpreterShowCreateAccessEntityQuery.h | 2 + .../InterpreterShowRowPoliciesQuery.cpp | 68 +++++ .../InterpreterShowRowPoliciesQuery.h | 25 ++ dbms/src/Interpreters/Users.cpp | 2 + dbms/src/Interpreters/Users.h | 1 + dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp | 164 +++++++++++ dbms/src/Parsers/ASTCreateRowPolicyQuery.h | 50 ++++ dbms/src/Parsers/ASTDropAccessEntityQuery.cpp | 32 ++- dbms/src/Parsers/ASTDropAccessEntityQuery.h | 4 + .../ASTShowCreateAccessEntityQuery.cpp | 12 +- .../Parsers/ASTShowCreateAccessEntityQuery.h | 4 + dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp | 22 ++ dbms/src/Parsers/ASTShowRowPoliciesQuery.h | 23 ++ .../Parsers/ParserCreateRowPolicyQuery.cpp | 261 ++++++++++++++++++ dbms/src/Parsers/ParserCreateRowPolicyQuery.h | 30 ++ .../Parsers/ParserDropAccessEntityQuery.cpp | 49 +++- dbms/src/Parsers/ParserQuery.cpp | 3 + dbms/src/Parsers/ParserQueryWithOutput.cpp | 5 +- .../ParserShowCreateAccessEntityQuery.cpp | 37 ++- .../Parsers/ParserShowRowPoliciesQuery.cpp | 40 +++ dbms/src/Parsers/ParserShowRowPoliciesQuery.h | 17 ++ .../tests/integration/test_row_policy/test.py | 54 ++++ .../01039_row_policy_dcl.reference | 0 .../0_stateless/01039_row_policy_dcl.sql | 2 + 31 files changed, 1089 insertions(+), 25 deletions(-) create mode 100644 dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h create mode 100644 dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h create mode 100644 dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp create mode 100644 dbms/src/Parsers/ASTCreateRowPolicyQuery.h create mode 100644 dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp create mode 100644 dbms/src/Parsers/ASTShowRowPoliciesQuery.h create mode 100644 dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp create mode 100644 dbms/src/Parsers/ParserCreateRowPolicyQuery.h create mode 100644 dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp create mode 100644 dbms/src/Parsers/ParserShowRowPoliciesQuery.h create mode 100644 dbms/tests/queries/0_stateless/01039_row_policy_dcl.reference create mode 100644 dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index e6b54f3dbc4..e5aab7075ca 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -627,6 +627,13 @@ void Context::checkQuotaManagementIsAllowed() "User " + client_info.current_user + " doesn't have enough privileges to manage quotas", ErrorCodes::NOT_ENOUGH_PRIVILEGES); } +void Context::checkRowPolicyManagementIsAllowed() +{ + if (!is_row_policy_management_allowed) + throw Exception( + "User " + client_info.current_user + " doesn't have enough privileges to manage row policies", ErrorCodes::NOT_ENOUGH_PRIVILEGES); +} + void Context::setUsersConfig(const ConfigurationPtr & config) { auto lock = getLock(); @@ -666,6 +673,7 @@ void Context::calculateUserSettings() client_info.current_user, client_info.current_address.host(), client_info.quota_key); is_quota_management_allowed = user->is_quota_management_allowed; row_policy = getAccessControlManager().getRowPolicyContext(client_info.current_user); + is_row_policy_management_allowed = user->is_row_policy_management_allowed; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index f434df05238..79e2936d7d2 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -142,6 +142,7 @@ private: std::shared_ptr quota; /// Current quota. By default - empty quota, that have no limits. bool is_quota_management_allowed = false; /// Whether the current user is allowed to manage quotas via SQL commands. std::shared_ptr row_policy; + bool is_row_policy_management_allowed = false; /// Whether the current user is allowed to manage row policies via SQL commands. String current_database; Settings settings; /// Setting for query execution. std::shared_ptr settings_constraints; @@ -212,6 +213,7 @@ public: std::shared_ptr getQuota() const { return quota; } void checkQuotaManagementIsAllowed(); std::shared_ptr getRowPolicy() const { return row_policy; } + void checkRowPolicyManagementIsAllowed(); /** Take the list of users, quotas and configuration profiles from this config. * The list of users is completely replaced. diff --git a/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp new file mode 100644 index 00000000000..daab0762826 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +BlockIO InterpreterCreateRowPolicyQuery::execute() +{ + context.checkRowPolicyManagementIsAllowed(); + const auto & query = query_ptr->as(); + auto & access_control = context.getAccessControlManager(); + + if (query.alter) + { + auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr + { + auto updated_policy = typeid_cast>(entity->clone()); + updateRowPolicyFromQuery(*updated_policy, query); + return updated_policy; + }; + String full_name = query.name_parts.getFullName(context); + if (query.if_exists) + { + if (auto id = access_control.find(full_name)) + access_control.tryUpdate(*id, update_func); + } + else + access_control.update(access_control.getID(full_name), update_func); + } + else + { + auto new_policy = std::make_shared(); + updateRowPolicyFromQuery(*new_policy, query); + + if (query.if_not_exists) + access_control.tryInsert(new_policy); + else if (query.or_replace) + access_control.insertOrReplace(new_policy); + else + access_control.insert(new_policy); + } + + return {}; +} + + +void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query) +{ + if (query.alter) + { + if (!query.new_policy_name.empty()) + policy.setName(query.new_policy_name); + } + else + { + policy.setDatabase(query.name_parts.database.empty() ? context.getCurrentDatabase() : query.name_parts.database); + policy.setTableName(query.name_parts.table_name); + policy.setName(query.name_parts.policy_name); + } + + if (query.is_restrictive) + policy.setRestrictive(*query.is_restrictive); + + for (const auto & [index, condition] : query.conditions) + policy.conditions[index] = condition ? serializeAST(*condition) : String{}; + + if (query.roles) + { + const auto & query_roles = *query.roles; + + /// We keep `roles` sorted. + policy.roles = query_roles.roles; + if (query_roles.current_user) + policy.roles.push_back(context.getClientInfo().current_user); + boost::range::sort(policy.roles); + policy.roles.erase(std::unique(policy.roles.begin(), policy.roles.end()), policy.roles.end()); + + policy.all_roles = query_roles.all_roles; + + /// We keep `except_roles` sorted. + policy.except_roles = query_roles.except_roles; + if (query_roles.except_current_user) + policy.except_roles.push_back(context.getClientInfo().current_user); + boost::range::sort(policy.except_roles); + policy.except_roles.erase(std::unique(policy.except_roles.begin(), policy.except_roles.end()), policy.except_roles.end()); + } +} +} diff --git a/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h new file mode 100644 index 00000000000..e7ee47dbe81 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTCreateRowPolicyQuery; +struct RowPolicy; + + +class InterpreterCreateRowPolicyQuery : public IInterpreter +{ +public: + InterpreterCreateRowPolicyQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + +private: + void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query); + + ASTPtr query_ptr; + Context & context; +}; +} diff --git a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp index 7f18084038c..c8e7ab43a64 100644 --- a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB @@ -24,6 +26,19 @@ BlockIO InterpreterDropAccessEntityQuery::execute() access_control.remove(access_control.getIDs(query.names)); return {}; } + case Kind::ROW_POLICY: + { + context.checkRowPolicyManagementIsAllowed(); + Strings full_names; + boost::range::transform( + query.row_policies_names, std::back_inserter(full_names), + [this](const RowPolicy::FullNameParts & row_policy_name) { return row_policy_name.getFullName(context); }); + if (query.if_exists) + access_control.tryRemove(access_control.find(full_names)); + else + access_control.remove(access_control.getIDs(full_names)); + return {}; + } } __builtin_unreachable(); diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 33e9da95dfc..7d1eccd8581 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -24,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -41,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -199,6 +203,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); @@ -211,6 +219,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); } diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index d0ef8992691..cf2a07425ba 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -1,9 +1,12 @@ #include #include #include +#include #include #include +#include #include +#include #include #include #include @@ -28,7 +31,7 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() const auto & show_query = query_ptr->as(); /// Build a create query. - ASTPtr create_query = getCreateQuotaQuery(show_query); + ASTPtr create_query = getCreateQuery(show_query); /// Build the result column. std::stringstream create_query_ss; @@ -49,6 +52,18 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() } +ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const +{ + using Kind = ASTShowCreateAccessEntityQuery::Kind; + switch (show_query.kind) + { + case Kind::QUOTA: return getCreateQuotaQuery(show_query); + case Kind::ROW_POLICY: return getCreateRowPolicyQuery(show_query); + } + __builtin_unreachable(); +} + + ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const { auto & access_control = context.getAccessControlManager(); @@ -86,4 +101,38 @@ ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShow return create_query; } + + +ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const +{ + auto & access_control = context.getAccessControlManager(); + RowPolicyPtr policy = access_control.read(show_query.row_policy_name.getFullName(context)); + + auto create_query = std::make_shared(); + create_query->name_parts = RowPolicy::FullNameParts{policy->getDatabase(), policy->getTableName(), policy->getName()}; + if (policy->isRestrictive()) + create_query->is_restrictive = policy->isRestrictive(); + + for (auto index : ext::range_with_static_cast(RowPolicy::MAX_CONDITION_INDEX)) + { + const auto & condition = policy->conditions[index]; + if (!condition.empty()) + { + ParserExpression parser; + ASTPtr expr = parseQuery(parser, condition, 0); + create_query->conditions.push_back(std::pair{index, expr}); + } + } + + if (!policy->roles.empty() || policy->all_roles) + { + auto create_query_roles = std::make_shared(); + create_query_roles->roles = policy->roles; + create_query_roles->all_roles = policy->all_roles; + create_query_roles->except_roles = policy->except_roles; + create_query->roles = std::move(create_query_roles); + } + + return create_query; +} } diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h index 94b06dadb19..0a6d94b85d0 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h @@ -28,7 +28,9 @@ private: const Context & context; BlockInputStreamPtr executeImpl(); + ASTPtr getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const; ASTPtr getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const; + ASTPtr getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const; }; diff --git a/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp new file mode 100644 index 00000000000..4b9b80d91a8 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +InterpreterShowRowPoliciesQuery::InterpreterShowRowPoliciesQuery(const ASTPtr & query_ptr_, Context & context_) + : query_ptr(query_ptr_), context(context_) +{ +} + + +BlockIO InterpreterShowRowPoliciesQuery::execute() +{ + return executeQuery(getRewrittenQuery(), context, true); +} + + +String InterpreterShowRowPoliciesQuery::getRewrittenQuery() const +{ + const auto & query = query_ptr->as(); + + const String & table_name = query.table_name; + String database; + if (!table_name.empty()) + { + database = query.database; + if (database.empty()) + database = context.getCurrentDatabase(); + } + + String filter; + if (query.current) + { + if (table_name.empty()) + filter = "has(currentRowPolicyIDs(), id)"; + else + filter = "has(currentRowPolicyIDs(" + quoteString(database) + ", " + quoteString(table_name) + "), id)"; + } + else + { + if (!table_name.empty()) + filter = "database = " + quoteString(database) + " AND table = " + quoteString(table_name); + } + + String expr = table_name.empty() ? "full_name" : "name"; + + return "SELECT " + expr + " AS " + backQuote(getResultDescription()) + " from system.row_policies" + + (filter.empty() ? "" : " WHERE " + filter) + " ORDER BY " + expr; +} + + +String InterpreterShowRowPoliciesQuery::getResultDescription() const +{ + std::stringstream ss; + formatAST(*query_ptr, ss, false, true); + String desc = ss.str(); + String prefix = "SHOW "; + if (startsWith(desc, prefix)) + desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. + return desc; +} +} diff --git a/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h new file mode 100644 index 00000000000..84cf6299b84 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowRowPoliciesQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class Context; + +class InterpreterShowRowPoliciesQuery : public IInterpreter +{ +public: + InterpreterShowRowPoliciesQuery(const ASTPtr & query_ptr_, Context & context_); + BlockIO execute() override; + +private: + String getRewrittenQuery() const; + String getResultDescription() const; + + ASTPtr query_ptr; + Context & context; +}; + +} diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index 81898ba5135..2b48ce56e3f 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -105,6 +105,8 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A if (config.has(config_elem + ".allow_quota_management")) is_quota_management_allowed = config.getBool(config_elem + ".allow_quota_management"); + if (config.has(config_elem + ".allow_row_policy_management")) + is_row_policy_management_allowed = config.getBool(config_elem + ".allow_row_policy_management"); } } diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index 189bfb991d5..f151770cefd 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -43,6 +43,7 @@ struct User std::optional dictionaries; bool is_quota_management_allowed = false; + bool is_row_policy_management_allowed = false; User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config); }; diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp new file mode 100644 index 00000000000..50e1645f14b --- /dev/null +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -0,0 +1,164 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + using ConditionIndex = RowPolicy::ConditionIndex; + + void formatRenameTo(const String & new_policy_name, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "") + << backQuote(new_policy_name); + } + + + void formatIsRestrictive(bool is_restrictive, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " AS " << (is_restrictive ? "RESTRICTIVE" : "PERMISSIVE") + << (settings.hilite ? IAST::hilite_none : ""); + } + + + void formatConditionalExpression(const ASTPtr & expr, const IAST::FormatSettings & settings) + { + if (!expr) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " NONE" << (settings.hilite ? IAST::hilite_none : ""); + return; + } + expr->format(settings); + } + + + std::vector> + conditionalExpressionsToStrings(const std::vector> & exprs, const IAST::FormatSettings & settings) + { + std::vector> result; + std::stringstream ss; + IAST::FormatSettings temp_settings(ss, settings); + boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair & in) + { + formatConditionalExpression(in.second, temp_settings); + auto out = std::pair{in.first, ss.str()}; + ss.str(""); + return out; + }); + return result; + } + + + void formatConditions(const char * op, const std::optional & filter, const std::optional & check, bool alter, const IAST::FormatSettings & settings) + { + if (op) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " FOR" << (settings.hilite ? IAST::hilite_none : ""); + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << ' ' << op << (settings.hilite ? IAST::hilite_none : ""); + } + + if (filter) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " USING " << (settings.hilite ? IAST::hilite_none : "") << *filter; + + if (check && (alter || (check != filter))) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " WITH CHECK " << (settings.hilite ? IAST::hilite_none : "") << *check; + } + + + void formatMultipleConditions(const std::vector> & conditions, bool alter, const IAST::FormatSettings & settings) + { + std::optional scond[RowPolicy::MAX_CONDITION_INDEX]; + for (const auto & [index, scondition] : conditionalExpressionsToStrings(conditions, settings)) + scond[index] = scondition; + + if ((scond[RowPolicy::SELECT_FILTER] == scond[RowPolicy::UPDATE_FILTER]) + && (scond[RowPolicy::UPDATE_FILTER] == scond[RowPolicy::DELETE_FILTER]) + && (scond[RowPolicy::INSERT_CHECK] == scond[RowPolicy::UPDATE_CHECK]) + && (scond[RowPolicy::SELECT_FILTER] || scond[RowPolicy::INSERT_CHECK])) + { + formatConditions(nullptr, scond[RowPolicy::SELECT_FILTER], scond[RowPolicy::INSERT_CHECK], alter, settings); + return; + } + + bool need_comma = false; + if (scond[RowPolicy::SELECT_FILTER]) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + formatConditions("SELECT", scond[RowPolicy::SELECT_FILTER], {}, alter, settings); + } + if (scond[RowPolicy::INSERT_CHECK]) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + formatConditions("INSERT", {}, scond[RowPolicy::INSERT_CHECK], alter, settings); + } + if (scond[RowPolicy::UPDATE_FILTER] || scond[RowPolicy::UPDATE_CHECK]) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + formatConditions("UPDATE", scond[RowPolicy::UPDATE_FILTER], scond[RowPolicy::UPDATE_CHECK], alter, settings); + } + if (scond[RowPolicy::DELETE_FILTER]) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + formatConditions("DELETE", scond[RowPolicy::DELETE_FILTER], {}, alter, settings); + } + } + + void formatRoles(const ASTRoleList & roles, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : ""); + roles.format(settings); + } +} + + +String ASTCreateRowPolicyQuery::getID(char) const +{ + return "CREATE POLICY or ALTER POLICY query"; +} + + +ASTPtr ASTCreateRowPolicyQuery::clone() const +{ + return std::make_shared(*this); +} + + +void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER POLICY" : "CREATE POLICY") + << (settings.hilite ? hilite_none : ""); + + if (if_exists) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : ""); + else if (if_not_exists) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (settings.hilite ? hilite_none : ""); + else if (or_replace) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); + + const String & database = name_parts.database; + const String & table_name = name_parts.table_name; + const String & policy_name = name_parts.policy_name; + settings.ostr << " " << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON " + << (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".") << table_name; + + if (!new_policy_name.empty()) + formatRenameTo(new_policy_name, settings); + + if (is_restrictive) + formatIsRestrictive(*is_restrictive, settings); + + formatMultipleConditions(conditions, alter, settings); + + if (roles) + formatRoles(*roles, settings); +} +} diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.h b/dbms/src/Parsers/ASTCreateRowPolicyQuery.h new file mode 100644 index 00000000000..a4caf1aeb85 --- /dev/null +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class ASTRoleList; + +/** CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] name ON [database.]table + * [AS {PERMISSIVE | RESTRICTIVE}] + * [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}] + * [USING condition] + * [WITH CHECK condition] [,...] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + * + * ALTER [ROW] POLICY [IF EXISTS] name ON [database.]table + * [RENAME TO new_name] + * [AS {PERMISSIVE | RESTRICTIVE}] + * [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}] + * [USING {condition | NONE}] + * [WITH CHECK {condition | NONE}] [,...] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + */ +class ASTCreateRowPolicyQuery : public IAST +{ +public: + bool alter = false; + + bool if_exists = false; + bool if_not_exists = false; + bool or_replace = false; + + RowPolicy::FullNameParts name_parts; + String new_policy_name; + + std::optional is_restrictive; + using ConditionIndex = RowPolicy::ConditionIndex; + std::vector> conditions; + + std::shared_ptr roles; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp index 80d69ed5316..b80dc47a9f7 100644 --- a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp @@ -13,6 +13,7 @@ namespace switch (kind) { case Kind::QUOTA: return "QUOTA"; + case Kind::ROW_POLICY: return "POLICY"; } __builtin_unreachable(); } @@ -44,13 +45,32 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma << (if_exists ? " IF EXISTS" : "") << (settings.hilite ? hilite_none : ""); - bool need_comma = false; - for (const auto & name : names) + if (kind == Kind::ROW_POLICY) { - if (need_comma) - settings.ostr << ','; - need_comma = true; - settings.ostr << ' ' << backQuoteIfNeed(name); + bool need_comma = false; + for (const auto & row_policy_name : row_policies_names) + { + if (need_comma) + settings.ostr << ','; + need_comma = true; + const String & database = row_policy_name.database; + const String & table_name = row_policy_name.table_name; + const String & policy_name = row_policy_name.policy_name; + settings.ostr << ' ' << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON " + << (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".") + << backQuoteIfNeed(table_name); + } + } + else + { + bool need_comma = false; + for (const auto & name : names) + { + if (need_comma) + settings.ostr << ','; + need_comma = true; + settings.ostr << ' ' << backQuoteIfNeed(name); + } } } } diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.h b/dbms/src/Parsers/ASTDropAccessEntityQuery.h index 91b76253db4..690a89995cf 100644 --- a/dbms/src/Parsers/ASTDropAccessEntityQuery.h +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.h @@ -1,12 +1,14 @@ #pragma once #include +#include namespace DB { /** DROP QUOTA [IF EXISTS] name [,...] + * DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] */ class ASTDropAccessEntityQuery : public IAST { @@ -14,11 +16,13 @@ public: enum class Kind { QUOTA, + ROW_POLICY, }; const Kind kind; const char * const keyword; bool if_exists = false; Strings names; + std::vector row_policies_names; ASTDropAccessEntityQuery(Kind kind_); String getID(char) const override; diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp index 8509a902014..caa9fb620e9 100644 --- a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp @@ -13,6 +13,7 @@ namespace switch (kind) { case Kind::QUOTA: return "QUOTA"; + case Kind::ROW_POLICY: return "POLICY"; } __builtin_unreachable(); } @@ -43,7 +44,16 @@ void ASTShowCreateAccessEntityQuery::formatQueryImpl(const FormatSettings & sett << "SHOW CREATE " << keyword << (settings.hilite ? hilite_none : ""); - if (current_quota) + if (kind == Kind::ROW_POLICY) + { + const String & database = row_policy_name.database; + const String & table_name = row_policy_name.table_name; + const String & policy_name = row_policy_name.policy_name; + settings.ostr << ' ' << backQuoteIfNeed(policy_name) << (settings.hilite ? hilite_keyword : "") << " ON " + << (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".") + << backQuoteIfNeed(table_name); + } + else if ((kind == Kind::QUOTA) && current_quota) settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : ""); else settings.ostr << " " << backQuoteIfNeed(name); diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h index 32c0ace101b..c19f00b1d6e 100644 --- a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h @@ -1,11 +1,13 @@ #pragma once #include +#include namespace DB { /** SHOW CREATE QUOTA [name | CURRENT] + * SHOW CREATE [ROW] POLICY name ON [database.]table */ class ASTShowCreateAccessEntityQuery : public ASTQueryWithOutput { @@ -13,11 +15,13 @@ public: enum class Kind { QUOTA, + ROW_POLICY, }; const Kind kind; const char * const keyword; String name; bool current_quota = false; + RowPolicy::FullNameParts row_policy_name; ASTShowCreateAccessEntityQuery(Kind kind_); String getID(char) const override; diff --git a/dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp b/dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp new file mode 100644 index 00000000000..15e0e81f218 --- /dev/null +++ b/dbms/src/Parsers/ASTShowRowPoliciesQuery.cpp @@ -0,0 +1,22 @@ +#include +#include + + +namespace DB +{ +void ASTShowRowPoliciesQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW POLICIES" << (settings.hilite ? hilite_none : ""); + + if (current) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : ""); + + if (!table_name.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : ""); + if (!database.empty()) + settings.ostr << backQuoteIfNeed(database) << "."; + settings.ostr << backQuoteIfNeed(table_name); + } +} +} diff --git a/dbms/src/Parsers/ASTShowRowPoliciesQuery.h b/dbms/src/Parsers/ASTShowRowPoliciesQuery.h new file mode 100644 index 00000000000..ce82902e96d --- /dev/null +++ b/dbms/src/Parsers/ASTShowRowPoliciesQuery.h @@ -0,0 +1,23 @@ +#pragma once + +#include + + +namespace DB +{ +/// SHOW [ROW] POLICIES [CURRENT] [ON [database.]table] +class ASTShowRowPoliciesQuery : public ASTQueryWithOutput +{ +public: + bool current = false; + String database; + String table_name; + + String getID(char) const override { return "SHOW POLICIES query"; } + ASTPtr clone() const override { return std::make_shared(*this); } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp new file mode 100644 index 00000000000..2778ddea93f --- /dev/null +++ b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp @@ -0,0 +1,261 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + + +namespace +{ + using ConditionIndex = RowPolicy::ConditionIndex; + + bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_policy_name, bool alter) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!new_policy_name.empty() || !alter) + return false; + + if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected)) + return false; + + return parseIdentifierOrStringLiteral(pos, expected, new_policy_name); + }); + } + + bool parseIsRestrictive(IParserBase::Pos & pos, Expected & expected, std::optional & is_restrictive) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (is_restrictive) + return false; + + if (!ParserKeyword{"AS"}.ignore(pos, expected)) + return false; + + if (ParserKeyword{"RESTRICTIVE"}.ignore(pos, expected)) + is_restrictive = true; + else if (ParserKeyword{"PERMISSIVE"}.ignore(pos, expected)) + is_restrictive = false; + else + return false; + + return true; + }); + } + + bool parseConditionalExpression(IParserBase::Pos & pos, Expected & expected, std::optional & expr) + { + if (ParserKeyword("NONE").ignore(pos, expected)) + { + expr = nullptr; + return true; + } + ParserExpression parser; + ASTPtr x; + if (parser.parse(pos, x, expected)) + { + expr = x; + return true; + } + expr.reset(); + return false; + } + + bool parseConditions(IParserBase::Pos & pos, Expected & expected, std::vector> & conditions, bool alter) + { + return IParserBase::wrapParseImpl(pos, [&] + { + static constexpr char select_op[] = "SELECT"; + static constexpr char insert_op[] = "INSERT"; + static constexpr char update_op[] = "UPDATE"; + static constexpr char delete_op[] = "DELETE"; + std::vector ops; + + bool keyword_for = false; + if (ParserKeyword{"FOR"}.ignore(pos, expected)) + { + keyword_for = true; + do + { + if (ParserKeyword{"SELECT"}.ignore(pos, expected)) + ops.push_back(select_op); + else if (ParserKeyword{"INSERT"}.ignore(pos, expected)) + ops.push_back(insert_op); + else if (ParserKeyword{"UPDATE"}.ignore(pos, expected)) + ops.push_back(update_op); + else if (ParserKeyword{"DELETE"}.ignore(pos, expected)) + ops.push_back(delete_op); + else if (ParserKeyword{"ALL"}.ignore(pos, expected)) + { + } + else + return false; + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + } + + if (ops.empty()) + { + ops.push_back(select_op); + ops.push_back(insert_op); + ops.push_back(update_op); + ops.push_back(delete_op); + } + + std::optional filter; + std::optional check; + bool keyword_using = false, keyword_with_check = false; + if (ParserKeyword{"USING"}.ignore(pos, expected)) + { + keyword_using = true; + if (!parseConditionalExpression(pos, expected, filter)) + return false; + } + if (ParserKeyword{"WITH CHECK"}.ignore(pos, expected)) + { + keyword_with_check = true; + if (!parseConditionalExpression(pos, expected, check)) + return false; + } + + if (!keyword_for && !keyword_using && !keyword_with_check) + return false; + + if (filter && !check && !alter) + check = filter; + + auto set_condition = [&](ConditionIndex index, const ASTPtr & condition) + { + auto it = std::find_if(conditions.begin(), conditions.end(), [index](const std::pair & element) + { + return element.first == index; + }); + if (it == conditions.end()) + it = conditions.insert(conditions.end(), std::pair{index, nullptr}); + it->second = condition; + }; + + for (const auto & op : ops) + { + if ((op == select_op) && filter) + set_condition(RowPolicy::SELECT_FILTER, *filter); + else if ((op == insert_op) && check) + set_condition(RowPolicy::INSERT_CHECK, *check); + else if (op == update_op) + { + if (filter) + set_condition(RowPolicy::UPDATE_FILTER, *filter); + if (check) + set_condition(RowPolicy::UPDATE_CHECK, *check); + } + else if ((op == delete_op) && filter) + set_condition(RowPolicy::DELETE_FILTER, *filter); + else + __builtin_unreachable(); + } + + return true; + }); + } + + bool parseMultipleConditions(IParserBase::Pos & pos, Expected & expected, std::vector> & conditions, bool alter) + { + return IParserBase::wrapParseImpl(pos, [&] + { + do + { + if (!parseConditions(pos, expected, conditions, alter)) + return false; + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + return true; + }); + } + + bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr & roles) + { + return IParserBase::wrapParseImpl(pos, [&] + { + ASTPtr node; + if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserRoleList{}.parse(pos, node, expected)) + return false; + + roles = std::static_pointer_cast(node); + return true; + }); + } +} + + +bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + bool alter; + if (ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) || ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected)) + alter = false; + else if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected)) + alter = true; + else + return false; + + bool if_exists = false; + bool if_not_exists = false; + bool or_replace = false; + if (alter) + { + if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected)) + if_exists = true; + } + else + { + if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected)) + if_not_exists = true; + else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected)) + or_replace = true; + } + + RowPolicy::FullNameParts name_parts; + String & database = name_parts.database; + String & table_name = name_parts.table_name; + String & policy_name = name_parts.policy_name; + if (!parseIdentifierOrStringLiteral(pos, expected, policy_name) || !ParserKeyword{"ON"}.ignore(pos, expected) + || !parseDatabaseAndTableName(pos, expected, database, table_name)) + return false; + + String new_policy_name; + std::optional is_restrictive; + std::vector> conditions; + std::shared_ptr roles; + + while (parseRenameTo(pos, expected, new_policy_name, alter) || parseIsRestrictive(pos, expected, is_restrictive) + || parseMultipleConditions(pos, expected, conditions, alter) || parseRoles(pos, expected, roles)) + ; + + auto query = std::make_shared(); + node = query; + + query->alter = alter; + query->if_exists = if_exists; + query->if_not_exists = if_not_exists; + query->or_replace = or_replace; + query->name_parts = std::move(name_parts); + query->new_policy_name = std::move(new_policy_name); + query->is_restrictive = is_restrictive; + query->conditions = std::move(conditions); + query->roles = std::move(roles); + + return true; +} +} diff --git a/dbms/src/Parsers/ParserCreateRowPolicyQuery.h b/dbms/src/Parsers/ParserCreateRowPolicyQuery.h new file mode 100644 index 00000000000..2ec7ff37c94 --- /dev/null +++ b/dbms/src/Parsers/ParserCreateRowPolicyQuery.h @@ -0,0 +1,30 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] name ON [database.]table + * [AS {PERMISSIVE | RESTRICTIVE}] + * [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}] + * [USING condition] + * [WITH CHECK condition] [,...] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + * + * ALTER [ROW] POLICY [IF EXISTS] name ON [database.]table + * [RENAME TO new_name] + * [AS {PERMISSIVE | RESTRICTIVE}] + * [FOR {SELECT | INSERT | UPDATE | DELETE | ALL}] + * [USING {condition | NONE}] + * [WITH CHECK {condition | NONE}] [,...] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + */ +class ParserCreateRowPolicyQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE ROW POLICY or ALTER ROW POLICY query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp index c6d5ff889fc..1c34fd6edb6 100644 --- a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp @@ -2,11 +2,30 @@ #include #include #include +#include #include namespace DB { +namespace +{ + bool parseNames(IParserBase::Pos & pos, Expected & expected, Strings & names) + { + do + { + String name; + if (!parseIdentifierOrStringLiteral(pos, expected, name)) + return false; + + names.push_back(std::move(name)); + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + return true; + } +} + + bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!ParserKeyword{"DROP"}.ignore(pos, expected)) @@ -16,6 +35,8 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & Kind kind; if (ParserKeyword{"QUOTA"}.ignore(pos, expected)) kind = Kind::QUOTA; + else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected)) + kind = Kind::ROW_POLICY; else return false; @@ -24,21 +45,35 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if_exists = true; Strings names; - do - { - String name; - if (!parseIdentifierOrStringLiteral(pos, expected, name)) - return false; + std::vector row_policies_names; - names.push_back(std::move(name)); + if (kind == Kind::ROW_POLICY) + { + do + { + Strings policy_names; + if (!parseNames(pos, expected, policy_names)) + return false; + String database, table_name; + if (!ParserKeyword{"ON"}.ignore(pos, expected) || !parseDatabaseAndTableName(pos, expected, database, table_name)) + return false; + for (const String & policy_name : policy_names) + row_policies_names.push_back({database, table_name, policy_name}); + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + } + else + { + if (!parseNames(pos, expected, names)) + return false; } - while (ParserToken{TokenType::Comma}.ignore(pos, expected)); auto query = std::make_shared(kind); node = query; query->if_exists = if_exists; query->names = std::move(names); + query->row_policies_names = std::move(row_policies_names); return true; } diff --git a/dbms/src/Parsers/ParserQuery.cpp b/dbms/src/Parsers/ParserQuery.cpp index a3bb652032e..d5002794734 100644 --- a/dbms/src/Parsers/ParserQuery.cpp +++ b/dbms/src/Parsers/ParserQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include @@ -25,6 +26,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserSetQuery set_p; ParserSystemQuery system_p; ParserCreateQuotaQuery create_quota_p; + ParserCreateRowPolicyQuery create_row_policy_p; ParserDropAccessEntityQuery drop_access_entity_p; bool res = query_with_output_p.parse(pos, node, expected) @@ -33,6 +35,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || set_p.parse(pos, node, expected) || system_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected) + || create_row_policy_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected); return res; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index d08ae984c90..31948194976 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -38,6 +39,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserWatchQuery watch_p; ParserShowCreateAccessEntityQuery show_create_access_entity_p; ParserShowQuotasQuery show_quotas_p; + ParserShowRowPoliciesQuery show_row_policies_p; ASTPtr query; @@ -66,7 +68,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || kill_query_p.parse(pos, query, expected) || optimize_p.parse(pos, query, expected) || watch_p.parse(pos, query, expected) - || show_quotas_p.parse(pos, query, expected); + || show_quotas_p.parse(pos, query, expected) + || show_row_policies_p.parse(pos, query, expected); if (!parsed) return false; diff --git a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp index 661330ffd0b..0caba5e0495 100644 --- a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB @@ -15,25 +17,41 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe Kind kind; if (ParserKeyword{"QUOTA"}.ignore(pos, expected)) kind = Kind::QUOTA; + else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected)) + kind = Kind::ROW_POLICY; else return false; String name; bool current_quota = false; + RowPolicy::FullNameParts row_policy_name; - if ((kind == Kind::QUOTA) && ParserKeyword{"CURRENT"}.ignore(pos, expected)) + if (kind == Kind::ROW_POLICY) { - /// SHOW CREATE QUOTA CURRENT - current_quota = true; - } - else if (parseIdentifierOrStringLiteral(pos, expected, name)) - { - /// SHOW CREATE QUOTA name + String & database = row_policy_name.database; + String & table_name = row_policy_name.table_name; + String & policy_name = row_policy_name.policy_name; + if (!parseIdentifierOrStringLiteral(pos, expected, policy_name) || !ParserKeyword{"ON"}.ignore(pos, expected) + || !parseDatabaseAndTableName(pos, expected, database, table_name)) + return false; } else { - /// SHOW CREATE QUOTA - current_quota = true; + assert(kind == Kind::QUOTA); + if (ParserKeyword{"CURRENT"}.ignore(pos, expected)) + { + /// SHOW CREATE QUOTA CURRENT + current_quota = true; + } + else if (parseIdentifierOrStringLiteral(pos, expected, name)) + { + /// SHOW CREATE QUOTA name + } + else + { + /// SHOW CREATE QUOTA + current_quota = true; + } } auto query = std::make_shared(kind); @@ -41,6 +59,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe query->name = std::move(name); query->current_quota = current_quota; + query->row_policy_name = std::move(row_policy_name); return true; } diff --git a/dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp b/dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp new file mode 100644 index 00000000000..b07e7a386ba --- /dev/null +++ b/dbms/src/Parsers/ParserShowRowPoliciesQuery.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool parseONDatabaseAndTableName(IParserBase::Pos & pos, Expected & expected, String & database, String & table_name) + { + return IParserBase::wrapParseImpl(pos, [&] + { + database.clear(); + table_name.clear(); + return ParserKeyword{"ON"}.ignore(pos, expected) && parseDatabaseAndTableName(pos, expected, database, table_name); + }); + } +} + + +bool ParserShowRowPoliciesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"SHOW POLICIES"}.ignore(pos, expected) && !ParserKeyword{"SHOW ROW POLICIES"}.ignore(pos, expected)) + return false; + + bool current = ParserKeyword{"CURRENT"}.ignore(pos, expected); + + String database, table_name; + parseONDatabaseAndTableName(pos, expected, database, table_name); + + auto query = std::make_shared(); + query->current = current; + query->database = std::move(database); + query->table_name = std::move(table_name); + node = query; + return true; +} +} diff --git a/dbms/src/Parsers/ParserShowRowPoliciesQuery.h b/dbms/src/Parsers/ParserShowRowPoliciesQuery.h new file mode 100644 index 00000000000..df7413fb604 --- /dev/null +++ b/dbms/src/Parsers/ParserShowRowPoliciesQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * SHOW [ROW] POLICIES [CURRENT] [ON [database.]table] + */ +class ParserShowRowPoliciesQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW POLICIES query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/dbms/tests/integration/test_row_policy/test.py b/dbms/tests/integration/test_row_policy/test.py index 3de63d56fa0..421a4b0510c 100644 --- a/dbms/tests/integration/test_row_policy/test.py +++ b/dbms/tests/integration/test_row_policy/test.py @@ -153,3 +153,57 @@ def test_introspection(): assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs('mydb', 'filtered_table3'), id) ORDER BY table, name") == policy3 assert instance.query("SELECT * from system.row_policies ORDER BY table, name") == policy1 + policy2 + policy3 assert instance.query("SELECT * from system.row_policies WHERE has(currentRowPolicyIDs(), id) ORDER BY table, name") == policy1 + policy2 + policy3 + + +def test_dcl_introspection(): + assert instance.query("SHOW POLICIES ON mydb.filtered_table1") == "default\n" + assert instance.query("SHOW POLICIES CURRENT ON mydb.filtered_table2") == "default\n" + assert instance.query("SHOW POLICIES") == "default ON mydb.filtered_table1\ndefault ON mydb.filtered_table2\ndefault ON mydb.filtered_table3\n" + assert instance.query("SHOW POLICIES CURRENT") == "default ON mydb.filtered_table1\ndefault ON mydb.filtered_table2\ndefault ON mydb.filtered_table3\n" + + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n" + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n" + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n" + + copy_policy_xml('all_rows.xml') + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE POLICY default ON mydb.filtered_table1 FOR SELECT USING 1 TO default\n" + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE POLICY default ON mydb.filtered_table2 FOR SELECT USING 1 TO default\n" + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE POLICY default ON mydb.filtered_table3 FOR SELECT USING 1 TO default\n" + + copy_policy_xml('no_rows.xml') + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table1") == "CREATE POLICY default ON mydb.filtered_table1 FOR SELECT USING NULL TO default\n" + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE POLICY default ON mydb.filtered_table2 FOR SELECT USING NULL TO default\n" + assert instance.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE POLICY default ON mydb.filtered_table3 FOR SELECT USING NULL TO default\n" + + copy_policy_xml('no_filters.xml') + assert instance.query("SHOW POLICIES") == "" + + +def test_dcl_management(): + copy_policy_xml('no_filters.xml') + assert instance.query("SHOW POLICIES") == "" + + instance.query("CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING ab") + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n" + + instance.query("ALTER POLICY pA ON mydb.filtered_table1 RENAME TO pB") + assert instance.query("SELECT * FROM mydb.filtered_table1") == "1\t0\n" + assert instance.query("SHOW POLICIES CURRENT ON mydb.filtered_table1") == "pB\n" + assert instance.query("SHOW CREATE POLICY pB ON mydb.filtered_table1") == "CREATE POLICY pB ON mydb.filtered_table1 FOR SELECT USING a > b TO default\n" + + instance.query("DROP POLICY pB ON mydb.filtered_table1") + assert instance.query("SELECT * FROM mydb.filtered_table1") == "0\t0\n0\t1\n1\t0\n1\t1\n" + assert instance.query("SHOW POLICIES") == "" + + +def test_users_xml_is_readonly(): + assert re.search("storage is readonly", instance.query_and_get_error("DROP POLICY default ON mydb.filtered_table1")) diff --git a/dbms/tests/queries/0_stateless/01039_row_policy_dcl.reference b/dbms/tests/queries/0_stateless/01039_row_policy_dcl.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql b/dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql new file mode 100644 index 00000000000..dc558a55b26 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01039_row_policy_dcl.sql @@ -0,0 +1,2 @@ +SHOW POLICIES; +CREATE POLICY p1 ON dummytable; -- { serverError 497 } From 2977289b57bb446bf112e65e26ba2a2b9d3f6a73 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Fri, 20 Dec 2019 04:30:51 +0100 Subject: [PATCH 10/66] update test --- .../test_dictionary_ddl_on_cluster/test.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py b/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py index d77ab2b842a..4296b497d15 100644 --- a/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py +++ b/dbms/tests/integration/test_dictionary_ddl_on_cluster/test.py @@ -1,3 +1,4 @@ +import time import pytest from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException @@ -43,9 +44,6 @@ def test_dictionary_ddl_on_cluster(started_cluster): assert node.query("SELECT count() from sometbl") == "1\n" assert node.query("SELECT dictGetString('default.somedict', 'value', toUInt64({}))".format(num)) == node.name + '\n' - instance = started_cluster.instances['ch1'] - started_cluster.ddl_check_query(instance, "SYSTEM RELOAD DICTIONARY ON CLUSTER 'cluster' default.somedict") - ch1.query("DETACH DICTIONARY default.somedict ON CLUSTER 'cluster'") for node in [ch1, ch2, ch3, ch4]: @@ -58,6 +56,16 @@ def test_dictionary_ddl_on_cluster(started_cluster): assert node.query("SELECT count() from sometbl") == "1\n" assert node.query("SELECT dictGetString('default.somedict', 'value', toUInt64({}))".format(num)) == node.name + '\n' + + for num, node in enumerate([ch1, ch2, ch3, ch4]): + node.query("ALTER TABLE sometbl UPDATE value = 'new_key' WHERE 1") + + ch1.query("SYSTEM RELOAD DICTIONARY ON CLUSTER 'cluster' `default.somedict`") + time.sleep(2) # SYSTEN RELOAD is a asynchronous query + + for num, node in enumerate([ch1, ch2, ch3, ch4]): + assert node.query("SELECT dictGetString('default.somedict', 'value', toUInt64({}))".format(num)) == 'new_key' + '\n' + ch1.query("DROP DICTIONARY default.somedict ON CLUSTER 'cluster'") for node in [ch1, ch2, ch3, ch4]: From 76869bb6953b034635555b450756cdfe0e9c15b7 Mon Sep 17 00:00:00 2001 From: maxim Date: Sat, 21 Dec 2019 07:30:55 +0300 Subject: [PATCH 11/66] arrow version updated to apache-arrow-0.15.1 --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 87ac6fddaf2..b789226ccb2 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 87ac6fddaf21d0b4ee8b8090533ff293db0da1b4 +Subproject commit b789226ccb2124285792107c758bb3b40b3d082a From 558324e7734b9e6569387992cd03a47fd52469f0 Mon Sep 17 00:00:00 2001 From: maxim Date: Sun, 22 Dec 2019 10:15:51 +0300 Subject: [PATCH 12/66] fixed reading of parquet files containing columns of type `list` --- contrib/arrow-cmake/CMakeLists.txt | 197 +- .../cpp/src/parquet/parquet_constants.cpp | 2 +- .../cpp/src/parquet/parquet_constants.h | 2 +- .../cpp/src/parquet/parquet_types.cpp | 2048 +++++++++-------- .../cpp/src/parquet/parquet_types.h | 375 +-- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 68 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 8 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 33 +- 8 files changed, 1575 insertions(+), 1158 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 3ba24f49498..59e27e221c6 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -1,3 +1,5 @@ +include(ExternalProject) + # === thrift set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp) @@ -70,6 +72,106 @@ add_custom_command(OUTPUT orc_proto.pb.h orc_proto.pb.cc --cpp_out="${CMAKE_CURRENT_BINARY_DIR}" "${PROTO_DIR}/orc_proto.proto") + +# === flatbuffers + +############################################################## +# fbs - Step 1: download flatbuffers and build flatc executable +# TODO get rid of downloads. Should work offline. +############################################################## +set(FLATBUFFERS_VERSION v1.11.0) #flatbuffers-v1.11.0.tar.gz + +if (DEFINED ENV{ARROW_FLATBUFFERS_URL}) + set(FLATBUFFERS_SOURCE_URL "$ENV{ARROW_FLATBUFFERS_URL}") +else() + set(FLATBUFFERS_SOURCE_URL "https://github.com/google/flatbuffers/archive/${FLATBUFFERS_VERSION}.tar.gz") +endif() + +macro(build_flatbuffers) + message(STATUS "Building flatbuffers from source") + set(FLATBUFFERS_PREFIX + "${CMAKE_CURRENT_BINARY_DIR}/flatbuffers_ep-prefix/src/flatbuffers_ep-install") + if(MSVC) + set(FLATBUFFERS_CMAKE_CXX_FLAGS /EHsc) + else() + set(FLATBUFFERS_CMAKE_CXX_FLAGS -fPIC) + endif() + set(FLATBUFFERS_COMPILER "${FLATBUFFERS_PREFIX}/bin/flatc") + set( + FLATBUFFERS_STATIC_LIB + "${FLATBUFFERS_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}flatbuffers${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) + # We always need to do release builds, otherwise flatc will not be installed. + externalproject_add(flatbuffers_ep + URL ${FLATBUFFERS_SOURCE_URL} + BUILD_BYPRODUCTS ${FLATBUFFERS_COMPILER} ${FLATBUFFERS_STATIC_LIB} + CMAKE_ARGS ${EP_COMMON_CMAKE_ARGS} + "-DCMAKE_BUILD_TYPE=RELEASE" + "-DCMAKE_CXX_FLAGS=${FLATBUFFERS_CMAKE_CXX_FLAGS}" + "-DCMAKE_INSTALL_PREFIX:PATH=${FLATBUFFERS_PREFIX}" + "-DFLATBUFFERS_BUILD_TESTS=OFF" + ${EP_LOG_OPTIONS}) + + file(MAKE_DIRECTORY "${FLATBUFFERS_PREFIX}/include") + + add_library(flatbuffers::flatbuffers STATIC IMPORTED) + set_target_properties(flatbuffers::flatbuffers + PROPERTIES IMPORTED_LOCATION "${FLATBUFFERS_STATIC_LIB}" + INTERFACE_INCLUDE_DIRECTORIES + "${FLATBUFFERS_PREFIX}/include") + add_executable(flatbuffers::flatc IMPORTED) + set_target_properties(flatbuffers::flatc + PROPERTIES IMPORTED_LOCATION "${FLATBUFFERS_COMPILER}") + + add_dependencies(flatbuffers::flatbuffers flatbuffers_ep) + add_dependencies(flatbuffers::flatc flatbuffers_ep) +endmacro() + +build_flatbuffers() + +################################### +# fbs - Step 2: compile *.fbs files +################################### +set(ARROW_IPC_SRC_DIR ${ARROW_SRC_DIR}/arrow/ipc) +set(ARROW_FORMAT_SRC_DIR ${ARROW_SRC_DIR}/../../format) + +set(FBS_OUTPUT_FILES "${ARROW_IPC_SRC_DIR}/File_generated.h" "${ARROW_IPC_SRC_DIR}/Message_generated.h" + "${ARROW_IPC_SRC_DIR}/feather_generated.h") + +set(FBS_SRC + ${ARROW_FORMAT_SRC_DIR}/Message.fbs + ${ARROW_FORMAT_SRC_DIR}/File.fbs + ${ARROW_FORMAT_SRC_DIR}/Schema.fbs + ${ARROW_FORMAT_SRC_DIR}/Tensor.fbs + ${ARROW_FORMAT_SRC_DIR}/SparseTensor.fbs + ${ARROW_IPC_SRC_DIR}/feather.fbs) + +foreach(FIL ${FBS_SRC}) + get_filename_component(ABS_FIL ${FIL} ABSOLUTE) + list(APPEND ABS_FBS_SRC ${ABS_FIL}) +endforeach() + +get_target_property(FLATC_EXECUTABLE flatbuffers::flatc IMPORTED_LOCATION) +if(NOT FLATC_EXECUTABLE) + get_target_property(FLATC_EXECUTABLE flatbuffers::flatc IMPORTED_LOCATION_RELEASE) +endif() + +# TODO compiled files should be stored in binary build dir instead of among submodule sources +set(FLATBUFFERS_COMPILED_OUT_DIR ${ARROW_IPC_SRC_DIR}) + +message(STATUS "flatc: ${FLATC_EXECUTABLE} -c -o ${FLATBUFFERS_COMPILED_OUT_DIR}/ ${ABS_FBS_SRC}") + +add_custom_command(TARGET flatbuffers_ep + POST_BUILD + COMMAND ${FLATC_EXECUTABLE} + -c + -o + ${FLATBUFFERS_COMPILED_OUT_DIR}/ + ${ABS_FBS_SRC} + DEPENDS flatbuffers::flatc ${ABS_FBS_SRC} + COMMENT "Running flatc compiler on ${ABS_FBS_SRC}" + VERBATIM) + # arrow-cmake cmake file calling orc cmake subroutine which detects certain compiler features. # Apple Clang compiler failed to compile this code without specifying c++11 standard. # As result these compiler features detected as absent. In result it failed to compile orc itself. @@ -86,6 +188,7 @@ configure_file("${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" "${ORC_BUILD_INCLUDE_DIR}/A set(ORC_SRCS ${ARROW_SRC_DIR}/arrow/adapters/orc/adapter.cc + ${ARROW_SRC_DIR}/arrow/adapters/orc/adapter_util.cc ${ORC_SOURCE_SRC_DIR}/Exceptions.cc ${ORC_SOURCE_SRC_DIR}/OrcFile.cc ${ORC_SOURCE_SRC_DIR}/Reader.cc @@ -119,11 +222,29 @@ set(ORC_SRCS # === arrow set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/arrow) + +configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${LIBRARY_DIR}/util/config.h") + # arrow/cpp/src/arrow/CMakeLists.txt set(ARROW_SRCS ${LIBRARY_DIR}/array.cc - + ${LIBRARY_DIR}/buffer.cc ${LIBRARY_DIR}/builder.cc + ${LIBRARY_DIR}/compare.cc + ${LIBRARY_DIR}/extension_type.cc + ${LIBRARY_DIR}/memory_pool.cc + ${LIBRARY_DIR}/pretty_print.cc + ${LIBRARY_DIR}/record_batch.cc + ${LIBRARY_DIR}/result.cc + ${LIBRARY_DIR}/scalar.cc + ${LIBRARY_DIR}/sparse_tensor.cc + ${LIBRARY_DIR}/status.cc + ${LIBRARY_DIR}/table_builder.cc + ${LIBRARY_DIR}/table.cc + ${LIBRARY_DIR}/tensor.cc + ${LIBRARY_DIR}/type.cc + ${LIBRARY_DIR}/visitor.cc + ${LIBRARY_DIR}/array/builder_adaptive.cc ${LIBRARY_DIR}/array/builder_base.cc ${LIBRARY_DIR}/array/builder_binary.cc @@ -131,46 +252,61 @@ set(ARROW_SRCS ${LIBRARY_DIR}/array/builder_dict.cc ${LIBRARY_DIR}/array/builder_nested.cc ${LIBRARY_DIR}/array/builder_primitive.cc - - ${LIBRARY_DIR}/buffer.cc - ${LIBRARY_DIR}/compare.cc - ${LIBRARY_DIR}/memory_pool.cc - ${LIBRARY_DIR}/pretty_print.cc - ${LIBRARY_DIR}/record_batch.cc - ${LIBRARY_DIR}/status.cc - ${LIBRARY_DIR}/table.cc - ${LIBRARY_DIR}/table_builder.cc - ${LIBRARY_DIR}/tensor.cc - ${LIBRARY_DIR}/sparse_tensor.cc - ${LIBRARY_DIR}/type.cc - ${LIBRARY_DIR}/visitor.cc + ${LIBRARY_DIR}/array/builder_union.cc + ${LIBRARY_DIR}/array/concatenate.cc + ${LIBRARY_DIR}/array/dict_internal.cc + ${LIBRARY_DIR}/array/diff.cc ${LIBRARY_DIR}/csv/converter.cc ${LIBRARY_DIR}/csv/chunker.cc - ${LIBRARY_DIR}/csv/column-builder.cc + ${LIBRARY_DIR}/csv/column_builder.cc ${LIBRARY_DIR}/csv/options.cc ${LIBRARY_DIR}/csv/parser.cc ${LIBRARY_DIR}/csv/reader.cc + ${LIBRARY_DIR}/ipc/dictionary.cc + ${LIBRARY_DIR}/ipc/feather.cc + ${LIBRARY_DIR}/ipc/file_to_stream.cc + ${LIBRARY_DIR}/ipc/message.cc + ${LIBRARY_DIR}/ipc/metadata_internal.cc + ${LIBRARY_DIR}/ipc/options.cc + ${LIBRARY_DIR}/ipc/reader.cc + ${LIBRARY_DIR}/ipc/stream_to_file.cc + ${LIBRARY_DIR}/ipc/writer.cc + ${LIBRARY_DIR}/io/buffered.cc ${LIBRARY_DIR}/io/compressed.cc ${LIBRARY_DIR}/io/file.cc ${LIBRARY_DIR}/io/interfaces.cc ${LIBRARY_DIR}/io/memory.cc ${LIBRARY_DIR}/io/readahead.cc + ${LIBRARY_DIR}/io/slow.cc - ${LIBRARY_DIR}/util/bit-util.cc + ${LIBRARY_DIR}/util/basic_decimal.cc + ${LIBRARY_DIR}/util/bit_util.cc +# ${LIBRARY_DIR}/util/compression_brotli.cc + ${LIBRARY_DIR}/util/compression_bz2.cc ${LIBRARY_DIR}/util/compression.cc - ${LIBRARY_DIR}/util/cpu-info.cc + ${LIBRARY_DIR}/util/compression_lz4.cc + ${LIBRARY_DIR}/util/compression_snappy.cc + ${LIBRARY_DIR}/util/compression_zlib.cc + ${LIBRARY_DIR}/util/compression_zstd.cc + ${LIBRARY_DIR}/util/cpu_info.cc ${LIBRARY_DIR}/util/decimal.cc - ${LIBRARY_DIR}/util/int-util.cc - ${LIBRARY_DIR}/util/io-util.cc - ${LIBRARY_DIR}/util/logging.cc + ${LIBRARY_DIR}/util/int_util.cc + ${LIBRARY_DIR}/util/io_util.cc ${LIBRARY_DIR}/util/key_value_metadata.cc - ${LIBRARY_DIR}/util/task-group.cc - ${LIBRARY_DIR}/util/thread-pool.cc + ${LIBRARY_DIR}/util/logging.cc + ${LIBRARY_DIR}/util/memory.cc + ${LIBRARY_DIR}/util/string_builder.cc + ${LIBRARY_DIR}/util/string.cc + ${LIBRARY_DIR}/util/task_group.cc + ${LIBRARY_DIR}/util/thread_pool.cc ${LIBRARY_DIR}/util/trie.cc +# ${LIBRARY_DIR}/util/uri.cc ${LIBRARY_DIR}/util/utf8.cc + + ${LIBRARY_DIR}/vendored/base64.cpp ${ORC_SRCS} ) @@ -179,7 +315,7 @@ set(ARROW_SRCS ${ARROW_SRCS} ${LIBRARY_DIR}/compute/kernels/boolean.cc ${LIBRARY_DIR}/compute/kernels/cast.cc ${LIBRARY_DIR}/compute/kernels/hash.cc - ${LIBRARY_DIR}/compute/kernels/util-internal.cc + ${LIBRARY_DIR}/compute/kernels/util_internal.cc ) if (LZ4_INCLUDE_DIR AND LZ4_LIBRARY) @@ -221,6 +357,11 @@ endif() add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) +# Arrow dependencies on external projects (ep): flatbuffers and Boost +add_dependencies(${ARROW_LIBRARY} flatbuffers_ep) +target_link_libraries(${ARROW_LIBRARY} PRIVATE boost_system_internal boost_filesystem_internal boost_regex_internal) +target_link_libraries(${ARROW_LIBRARY} PRIVATE flatbuffers::flatbuffers) + if (USE_INTERNAL_PROTOBUF_LIBRARY) add_dependencies(${ARROW_LIBRARY} protoc) endif() @@ -255,23 +396,25 @@ set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/parquet) # arrow/cpp/src/parquet/CMakeLists.txt set(PARQUET_SRCS ${LIBRARY_DIR}/arrow/reader.cc - ${LIBRARY_DIR}/arrow/record_reader.cc + ${LIBRARY_DIR}/arrow/reader_internal.cc ${LIBRARY_DIR}/arrow/schema.cc ${LIBRARY_DIR}/arrow/writer.cc ${LIBRARY_DIR}/bloom_filter.cc ${LIBRARY_DIR}/column_reader.cc ${LIBRARY_DIR}/column_scanner.cc ${LIBRARY_DIR}/column_writer.cc + ${LIBRARY_DIR}/deprecated_io.cc + ${LIBRARY_DIR}/encoding.cc ${LIBRARY_DIR}/file_reader.cc ${LIBRARY_DIR}/file_writer.cc ${LIBRARY_DIR}/metadata.cc ${LIBRARY_DIR}/murmur3.cc + ${LIBRARY_DIR}/platform.cc ${LIBRARY_DIR}/printer.cc + ${LIBRARY_DIR}/properties.cc ${LIBRARY_DIR}/schema.cc ${LIBRARY_DIR}/statistics.cc ${LIBRARY_DIR}/types.cc - ${LIBRARY_DIR}/util/comparison.cc - ${LIBRARY_DIR}/util/memory.cc ) #list(TRANSFORM PARQUET_SRCS PREPEND ${LIBRARY_DIR}/) # cmake 3.12 list(APPEND PARQUET_SRCS @@ -292,7 +435,7 @@ endif() # === tools set(TOOLS_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/tools/parquet) -set(PARQUET_TOOLS parquet-dump-schema parquet-reader parquet-scan) +set(PARQUET_TOOLS parquet_dump_schema parquet_reader parquet_scan) foreach(TOOL ${PARQUET_TOOLS}) add_executable(${TOOL} ${TOOLS_DIR}/${TOOL}.cc) target_link_libraries(${TOOL} PRIVATE ${PARQUET_LIBRARY}) diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp b/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp index e1775208368..8efba22eb0e 100644 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp +++ b/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.11.0) + * Autogenerated by Thrift Compiler (0.12.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h b/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h index b4fbf3ab99e..a84198bfa44 100644 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h +++ b/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.11.0) + * Autogenerated by Thrift Compiler (0.12.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp b/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp index f9780d268f3..9ad74689567 100644 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp +++ b/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.11.0) + * Autogenerated by Thrift Compiler (0.12.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -1167,6 +1167,74 @@ void MicroSeconds::printTo(std::ostream& out) const { } +NanoSeconds::~NanoSeconds() throw() { +} + +std::ostream& operator<<(std::ostream& out, const NanoSeconds& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NanoSeconds::read(::apache::thrift::protocol::TProtocol* iprot) { + + ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + xfer += iprot->skip(ftype); + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t NanoSeconds::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NanoSeconds"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NanoSeconds &a, NanoSeconds &b) { + using ::std::swap; + (void) a; + (void) b; +} + +NanoSeconds::NanoSeconds(const NanoSeconds& other22) { + (void) other22; +} +NanoSeconds& NanoSeconds::operator=(const NanoSeconds& other23) { + (void) other23; + return *this; +} +void NanoSeconds::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NanoSeconds("; + out << ")"; +} + + TimeUnit::~TimeUnit() throw() { } @@ -1180,6 +1248,11 @@ void TimeUnit::__set_MICROS(const MicroSeconds& val) { this->MICROS = val; __isset.MICROS = true; } + +void TimeUnit::__set_NANOS(const NanoSeconds& val) { + this->NANOS = val; +__isset.NANOS = true; +} std::ostream& operator<<(std::ostream& out, const TimeUnit& obj) { obj.printTo(out); @@ -1224,6 +1297,14 @@ uint32_t TimeUnit::read(::apache::thrift::protocol::TProtocol* iprot) { xfer += iprot->skip(ftype); } break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->NANOS.read(iprot); + this->__isset.NANOS = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -1251,6 +1332,11 @@ uint32_t TimeUnit::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += this->MICROS.write(oprot); xfer += oprot->writeFieldEnd(); } + if (this->__isset.NANOS) { + xfer += oprot->writeFieldBegin("NANOS", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->NANOS.write(oprot); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -1260,18 +1346,21 @@ void swap(TimeUnit &a, TimeUnit &b) { using ::std::swap; swap(a.MILLIS, b.MILLIS); swap(a.MICROS, b.MICROS); + swap(a.NANOS, b.NANOS); swap(a.__isset, b.__isset); } -TimeUnit::TimeUnit(const TimeUnit& other22) { - MILLIS = other22.MILLIS; - MICROS = other22.MICROS; - __isset = other22.__isset; +TimeUnit::TimeUnit(const TimeUnit& other24) { + MILLIS = other24.MILLIS; + MICROS = other24.MICROS; + NANOS = other24.NANOS; + __isset = other24.__isset; } -TimeUnit& TimeUnit::operator=(const TimeUnit& other23) { - MILLIS = other23.MILLIS; - MICROS = other23.MICROS; - __isset = other23.__isset; +TimeUnit& TimeUnit::operator=(const TimeUnit& other25) { + MILLIS = other25.MILLIS; + MICROS = other25.MICROS; + NANOS = other25.NANOS; + __isset = other25.__isset; return *this; } void TimeUnit::printTo(std::ostream& out) const { @@ -1279,6 +1368,7 @@ void TimeUnit::printTo(std::ostream& out) const { out << "TimeUnit("; out << "MILLIS="; (__isset.MILLIS ? (out << to_string(MILLIS)) : (out << "")); out << ", " << "MICROS="; (__isset.MICROS ? (out << to_string(MICROS)) : (out << "")); + out << ", " << "NANOS="; (__isset.NANOS ? (out << to_string(NANOS)) : (out << "")); out << ")"; } @@ -1380,13 +1470,13 @@ void swap(TimestampType &a, TimestampType &b) { swap(a.unit, b.unit); } -TimestampType::TimestampType(const TimestampType& other24) { - isAdjustedToUTC = other24.isAdjustedToUTC; - unit = other24.unit; +TimestampType::TimestampType(const TimestampType& other26) { + isAdjustedToUTC = other26.isAdjustedToUTC; + unit = other26.unit; } -TimestampType& TimestampType::operator=(const TimestampType& other25) { - isAdjustedToUTC = other25.isAdjustedToUTC; - unit = other25.unit; +TimestampType& TimestampType::operator=(const TimestampType& other27) { + isAdjustedToUTC = other27.isAdjustedToUTC; + unit = other27.unit; return *this; } void TimestampType::printTo(std::ostream& out) const { @@ -1495,13 +1585,13 @@ void swap(TimeType &a, TimeType &b) { swap(a.unit, b.unit); } -TimeType::TimeType(const TimeType& other26) { - isAdjustedToUTC = other26.isAdjustedToUTC; - unit = other26.unit; +TimeType::TimeType(const TimeType& other28) { + isAdjustedToUTC = other28.isAdjustedToUTC; + unit = other28.unit; } -TimeType& TimeType::operator=(const TimeType& other27) { - isAdjustedToUTC = other27.isAdjustedToUTC; - unit = other27.unit; +TimeType& TimeType::operator=(const TimeType& other29) { + isAdjustedToUTC = other29.isAdjustedToUTC; + unit = other29.unit; return *this; } void TimeType::printTo(std::ostream& out) const { @@ -1610,13 +1700,13 @@ void swap(IntType &a, IntType &b) { swap(a.isSigned, b.isSigned); } -IntType::IntType(const IntType& other28) { - bitWidth = other28.bitWidth; - isSigned = other28.isSigned; +IntType::IntType(const IntType& other30) { + bitWidth = other30.bitWidth; + isSigned = other30.isSigned; } -IntType& IntType::operator=(const IntType& other29) { - bitWidth = other29.bitWidth; - isSigned = other29.isSigned; +IntType& IntType::operator=(const IntType& other31) { + bitWidth = other31.bitWidth; + isSigned = other31.isSigned; return *this; } void IntType::printTo(std::ostream& out) const { @@ -1682,11 +1772,11 @@ void swap(JsonType &a, JsonType &b) { (void) b; } -JsonType::JsonType(const JsonType& other30) { - (void) other30; +JsonType::JsonType(const JsonType& other32) { + (void) other32; } -JsonType& JsonType::operator=(const JsonType& other31) { - (void) other31; +JsonType& JsonType::operator=(const JsonType& other33) { + (void) other33; return *this; } void JsonType::printTo(std::ostream& out) const { @@ -1750,11 +1840,11 @@ void swap(BsonType &a, BsonType &b) { (void) b; } -BsonType::BsonType(const BsonType& other32) { - (void) other32; +BsonType::BsonType(const BsonType& other34) { + (void) other34; } -BsonType& BsonType::operator=(const BsonType& other33) { - (void) other33; +BsonType& BsonType::operator=(const BsonType& other35) { + (void) other35; return *this; } void BsonType::printTo(std::ostream& out) const { @@ -1827,6 +1917,11 @@ void LogicalType::__set_BSON(const BsonType& val) { this->BSON = val; __isset.BSON = true; } + +void LogicalType::__set_UUID(const UUIDType& val) { + this->UUID = val; +__isset.UUID = true; +} std::ostream& operator<<(std::ostream& out, const LogicalType& obj) { obj.printTo(out); @@ -1951,6 +2046,14 @@ uint32_t LogicalType::read(::apache::thrift::protocol::TProtocol* iprot) { xfer += iprot->skip(ftype); } break; + case 14: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->UUID.read(iprot); + this->__isset.UUID = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -2028,6 +2131,11 @@ uint32_t LogicalType::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += this->BSON.write(oprot); xfer += oprot->writeFieldEnd(); } + if (this->__isset.UUID) { + xfer += oprot->writeFieldBegin("UUID", ::apache::thrift::protocol::T_STRUCT, 14); + xfer += this->UUID.write(oprot); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -2047,38 +2155,41 @@ void swap(LogicalType &a, LogicalType &b) { swap(a.UNKNOWN, b.UNKNOWN); swap(a.JSON, b.JSON); swap(a.BSON, b.BSON); + swap(a.UUID, b.UUID); swap(a.__isset, b.__isset); } -LogicalType::LogicalType(const LogicalType& other34) { - STRING = other34.STRING; - MAP = other34.MAP; - LIST = other34.LIST; - ENUM = other34.ENUM; - DECIMAL = other34.DECIMAL; - DATE = other34.DATE; - TIME = other34.TIME; - TIMESTAMP = other34.TIMESTAMP; - INTEGER = other34.INTEGER; - UNKNOWN = other34.UNKNOWN; - JSON = other34.JSON; - BSON = other34.BSON; - __isset = other34.__isset; +LogicalType::LogicalType(const LogicalType& other36) { + STRING = other36.STRING; + MAP = other36.MAP; + LIST = other36.LIST; + ENUM = other36.ENUM; + DECIMAL = other36.DECIMAL; + DATE = other36.DATE; + TIME = other36.TIME; + TIMESTAMP = other36.TIMESTAMP; + INTEGER = other36.INTEGER; + UNKNOWN = other36.UNKNOWN; + JSON = other36.JSON; + BSON = other36.BSON; + UUID = other36.UUID; + __isset = other36.__isset; } -LogicalType& LogicalType::operator=(const LogicalType& other35) { - STRING = other35.STRING; - MAP = other35.MAP; - LIST = other35.LIST; - ENUM = other35.ENUM; - DECIMAL = other35.DECIMAL; - DATE = other35.DATE; - TIME = other35.TIME; - TIMESTAMP = other35.TIMESTAMP; - INTEGER = other35.INTEGER; - UNKNOWN = other35.UNKNOWN; - JSON = other35.JSON; - BSON = other35.BSON; - __isset = other35.__isset; +LogicalType& LogicalType::operator=(const LogicalType& other37) { + STRING = other37.STRING; + MAP = other37.MAP; + LIST = other37.LIST; + ENUM = other37.ENUM; + DECIMAL = other37.DECIMAL; + DATE = other37.DATE; + TIME = other37.TIME; + TIMESTAMP = other37.TIMESTAMP; + INTEGER = other37.INTEGER; + UNKNOWN = other37.UNKNOWN; + JSON = other37.JSON; + BSON = other37.BSON; + UUID = other37.UUID; + __isset = other37.__isset; return *this; } void LogicalType::printTo(std::ostream& out) const { @@ -2096,6 +2207,7 @@ void LogicalType::printTo(std::ostream& out) const { out << ", " << "UNKNOWN="; (__isset.UNKNOWN ? (out << to_string(UNKNOWN)) : (out << "")); out << ", " << "JSON="; (__isset.JSON ? (out << to_string(JSON)) : (out << "")); out << ", " << "BSON="; (__isset.BSON ? (out << to_string(BSON)) : (out << "")); + out << ", " << "UUID="; (__isset.UUID ? (out << to_string(UUID)) : (out << "")); out << ")"; } @@ -2183,9 +2295,9 @@ uint32_t SchemaElement::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast36; - xfer += iprot->readI32(ecast36); - this->type = (Type::type)ecast36; + int32_t ecast38; + xfer += iprot->readI32(ecast38); + this->type = (Type::type)ecast38; this->__isset.type = true; } else { xfer += iprot->skip(ftype); @@ -2201,9 +2313,9 @@ uint32_t SchemaElement::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast37; - xfer += iprot->readI32(ecast37); - this->repetition_type = (FieldRepetitionType::type)ecast37; + int32_t ecast39; + xfer += iprot->readI32(ecast39); + this->repetition_type = (FieldRepetitionType::type)ecast39; this->__isset.repetition_type = true; } else { xfer += iprot->skip(ftype); @@ -2227,9 +2339,9 @@ uint32_t SchemaElement::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 6: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast38; - xfer += iprot->readI32(ecast38); - this->converted_type = (ConvertedType::type)ecast38; + int32_t ecast40; + xfer += iprot->readI32(ecast40); + this->converted_type = (ConvertedType::type)ecast40; this->__isset.converted_type = true; } else { xfer += iprot->skip(ftype); @@ -2355,31 +2467,31 @@ void swap(SchemaElement &a, SchemaElement &b) { swap(a.__isset, b.__isset); } -SchemaElement::SchemaElement(const SchemaElement& other39) { - type = other39.type; - type_length = other39.type_length; - repetition_type = other39.repetition_type; - name = other39.name; - num_children = other39.num_children; - converted_type = other39.converted_type; - scale = other39.scale; - precision = other39.precision; - field_id = other39.field_id; - logicalType = other39.logicalType; - __isset = other39.__isset; +SchemaElement::SchemaElement(const SchemaElement& other41) { + type = other41.type; + type_length = other41.type_length; + repetition_type = other41.repetition_type; + name = other41.name; + num_children = other41.num_children; + converted_type = other41.converted_type; + scale = other41.scale; + precision = other41.precision; + field_id = other41.field_id; + logicalType = other41.logicalType; + __isset = other41.__isset; } -SchemaElement& SchemaElement::operator=(const SchemaElement& other40) { - type = other40.type; - type_length = other40.type_length; - repetition_type = other40.repetition_type; - name = other40.name; - num_children = other40.num_children; - converted_type = other40.converted_type; - scale = other40.scale; - precision = other40.precision; - field_id = other40.field_id; - logicalType = other40.logicalType; - __isset = other40.__isset; +SchemaElement& SchemaElement::operator=(const SchemaElement& other42) { + type = other42.type; + type_length = other42.type_length; + repetition_type = other42.repetition_type; + name = other42.name; + num_children = other42.num_children; + converted_type = other42.converted_type; + scale = other42.scale; + precision = other42.precision; + field_id = other42.field_id; + logicalType = other42.logicalType; + __isset = other42.__isset; return *this; } void SchemaElement::printTo(std::ostream& out) const { @@ -2465,9 +2577,9 @@ uint32_t DataPageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast41; - xfer += iprot->readI32(ecast41); - this->encoding = (Encoding::type)ecast41; + int32_t ecast43; + xfer += iprot->readI32(ecast43); + this->encoding = (Encoding::type)ecast43; isset_encoding = true; } else { xfer += iprot->skip(ftype); @@ -2475,9 +2587,9 @@ uint32_t DataPageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast42; - xfer += iprot->readI32(ecast42); - this->definition_level_encoding = (Encoding::type)ecast42; + int32_t ecast44; + xfer += iprot->readI32(ecast44); + this->definition_level_encoding = (Encoding::type)ecast44; isset_definition_level_encoding = true; } else { xfer += iprot->skip(ftype); @@ -2485,9 +2597,9 @@ uint32_t DataPageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast43; - xfer += iprot->readI32(ecast43); - this->repetition_level_encoding = (Encoding::type)ecast43; + int32_t ecast45; + xfer += iprot->readI32(ecast45); + this->repetition_level_encoding = (Encoding::type)ecast45; isset_repetition_level_encoding = true; } else { xfer += iprot->skip(ftype); @@ -2562,21 +2674,21 @@ void swap(DataPageHeader &a, DataPageHeader &b) { swap(a.__isset, b.__isset); } -DataPageHeader::DataPageHeader(const DataPageHeader& other44) { - num_values = other44.num_values; - encoding = other44.encoding; - definition_level_encoding = other44.definition_level_encoding; - repetition_level_encoding = other44.repetition_level_encoding; - statistics = other44.statistics; - __isset = other44.__isset; +DataPageHeader::DataPageHeader(const DataPageHeader& other46) { + num_values = other46.num_values; + encoding = other46.encoding; + definition_level_encoding = other46.definition_level_encoding; + repetition_level_encoding = other46.repetition_level_encoding; + statistics = other46.statistics; + __isset = other46.__isset; } -DataPageHeader& DataPageHeader::operator=(const DataPageHeader& other45) { - num_values = other45.num_values; - encoding = other45.encoding; - definition_level_encoding = other45.definition_level_encoding; - repetition_level_encoding = other45.repetition_level_encoding; - statistics = other45.statistics; - __isset = other45.__isset; +DataPageHeader& DataPageHeader::operator=(const DataPageHeader& other47) { + num_values = other47.num_values; + encoding = other47.encoding; + definition_level_encoding = other47.definition_level_encoding; + repetition_level_encoding = other47.repetition_level_encoding; + statistics = other47.statistics; + __isset = other47.__isset; return *this; } void DataPageHeader::printTo(std::ostream& out) const { @@ -2645,11 +2757,11 @@ void swap(IndexPageHeader &a, IndexPageHeader &b) { (void) b; } -IndexPageHeader::IndexPageHeader(const IndexPageHeader& other46) { - (void) other46; +IndexPageHeader::IndexPageHeader(const IndexPageHeader& other48) { + (void) other48; } -IndexPageHeader& IndexPageHeader::operator=(const IndexPageHeader& other47) { - (void) other47; +IndexPageHeader& IndexPageHeader::operator=(const IndexPageHeader& other49) { + (void) other49; return *this; } void IndexPageHeader::printTo(std::ostream& out) const { @@ -2715,9 +2827,9 @@ uint32_t DictionaryPageHeader::read(::apache::thrift::protocol::TProtocol* iprot break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast48; - xfer += iprot->readI32(ecast48); - this->encoding = (Encoding::type)ecast48; + int32_t ecast50; + xfer += iprot->readI32(ecast50); + this->encoding = (Encoding::type)ecast50; isset_encoding = true; } else { xfer += iprot->skip(ftype); @@ -2778,17 +2890,17 @@ void swap(DictionaryPageHeader &a, DictionaryPageHeader &b) { swap(a.__isset, b.__isset); } -DictionaryPageHeader::DictionaryPageHeader(const DictionaryPageHeader& other49) { - num_values = other49.num_values; - encoding = other49.encoding; - is_sorted = other49.is_sorted; - __isset = other49.__isset; +DictionaryPageHeader::DictionaryPageHeader(const DictionaryPageHeader& other51) { + num_values = other51.num_values; + encoding = other51.encoding; + is_sorted = other51.is_sorted; + __isset = other51.__isset; } -DictionaryPageHeader& DictionaryPageHeader::operator=(const DictionaryPageHeader& other50) { - num_values = other50.num_values; - encoding = other50.encoding; - is_sorted = other50.is_sorted; - __isset = other50.__isset; +DictionaryPageHeader& DictionaryPageHeader::operator=(const DictionaryPageHeader& other52) { + num_values = other52.num_values; + encoding = other52.encoding; + is_sorted = other52.is_sorted; + __isset = other52.__isset; return *this; } void DictionaryPageHeader::printTo(std::ostream& out) const { @@ -2898,9 +3010,9 @@ uint32_t DataPageHeaderV2::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast51; - xfer += iprot->readI32(ecast51); - this->encoding = (Encoding::type)ecast51; + int32_t ecast53; + xfer += iprot->readI32(ecast53); + this->encoding = (Encoding::type)ecast53; isset_encoding = true; } else { xfer += iprot->skip(ftype); @@ -3019,27 +3131,27 @@ void swap(DataPageHeaderV2 &a, DataPageHeaderV2 &b) { swap(a.__isset, b.__isset); } -DataPageHeaderV2::DataPageHeaderV2(const DataPageHeaderV2& other52) { - num_values = other52.num_values; - num_nulls = other52.num_nulls; - num_rows = other52.num_rows; - encoding = other52.encoding; - definition_levels_byte_length = other52.definition_levels_byte_length; - repetition_levels_byte_length = other52.repetition_levels_byte_length; - is_compressed = other52.is_compressed; - statistics = other52.statistics; - __isset = other52.__isset; +DataPageHeaderV2::DataPageHeaderV2(const DataPageHeaderV2& other54) { + num_values = other54.num_values; + num_nulls = other54.num_nulls; + num_rows = other54.num_rows; + encoding = other54.encoding; + definition_levels_byte_length = other54.definition_levels_byte_length; + repetition_levels_byte_length = other54.repetition_levels_byte_length; + is_compressed = other54.is_compressed; + statistics = other54.statistics; + __isset = other54.__isset; } -DataPageHeaderV2& DataPageHeaderV2::operator=(const DataPageHeaderV2& other53) { - num_values = other53.num_values; - num_nulls = other53.num_nulls; - num_rows = other53.num_rows; - encoding = other53.encoding; - definition_levels_byte_length = other53.definition_levels_byte_length; - repetition_levels_byte_length = other53.repetition_levels_byte_length; - is_compressed = other53.is_compressed; - statistics = other53.statistics; - __isset = other53.__isset; +DataPageHeaderV2& DataPageHeaderV2::operator=(const DataPageHeaderV2& other55) { + num_values = other55.num_values; + num_nulls = other55.num_nulls; + num_rows = other55.num_rows; + encoding = other55.encoding; + definition_levels_byte_length = other55.definition_levels_byte_length; + repetition_levels_byte_length = other55.repetition_levels_byte_length; + is_compressed = other55.is_compressed; + statistics = other55.statistics; + __isset = other55.__isset; return *this; } void DataPageHeaderV2::printTo(std::ostream& out) const { @@ -3130,9 +3242,9 @@ uint32_t PageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast54; - xfer += iprot->readI32(ecast54); - this->type = (PageType::type)ecast54; + int32_t ecast56; + xfer += iprot->readI32(ecast56); + this->type = (PageType::type)ecast56; isset_type = true; } else { xfer += iprot->skip(ftype); @@ -3272,27 +3384,27 @@ void swap(PageHeader &a, PageHeader &b) { swap(a.__isset, b.__isset); } -PageHeader::PageHeader(const PageHeader& other55) { - type = other55.type; - uncompressed_page_size = other55.uncompressed_page_size; - compressed_page_size = other55.compressed_page_size; - crc = other55.crc; - data_page_header = other55.data_page_header; - index_page_header = other55.index_page_header; - dictionary_page_header = other55.dictionary_page_header; - data_page_header_v2 = other55.data_page_header_v2; - __isset = other55.__isset; +PageHeader::PageHeader(const PageHeader& other57) { + type = other57.type; + uncompressed_page_size = other57.uncompressed_page_size; + compressed_page_size = other57.compressed_page_size; + crc = other57.crc; + data_page_header = other57.data_page_header; + index_page_header = other57.index_page_header; + dictionary_page_header = other57.dictionary_page_header; + data_page_header_v2 = other57.data_page_header_v2; + __isset = other57.__isset; } -PageHeader& PageHeader::operator=(const PageHeader& other56) { - type = other56.type; - uncompressed_page_size = other56.uncompressed_page_size; - compressed_page_size = other56.compressed_page_size; - crc = other56.crc; - data_page_header = other56.data_page_header; - index_page_header = other56.index_page_header; - dictionary_page_header = other56.dictionary_page_header; - data_page_header_v2 = other56.data_page_header_v2; - __isset = other56.__isset; +PageHeader& PageHeader::operator=(const PageHeader& other58) { + type = other58.type; + uncompressed_page_size = other58.uncompressed_page_size; + compressed_page_size = other58.compressed_page_size; + crc = other58.crc; + data_page_header = other58.data_page_header; + index_page_header = other58.index_page_header; + dictionary_page_header = other58.dictionary_page_header; + data_page_header_v2 = other58.data_page_header_v2; + __isset = other58.__isset; return *this; } void PageHeader::printTo(std::ostream& out) const { @@ -3407,15 +3519,15 @@ void swap(KeyValue &a, KeyValue &b) { swap(a.__isset, b.__isset); } -KeyValue::KeyValue(const KeyValue& other57) { - key = other57.key; - value = other57.value; - __isset = other57.__isset; +KeyValue::KeyValue(const KeyValue& other59) { + key = other59.key; + value = other59.value; + __isset = other59.__isset; } -KeyValue& KeyValue::operator=(const KeyValue& other58) { - key = other58.key; - value = other58.value; - __isset = other58.__isset; +KeyValue& KeyValue::operator=(const KeyValue& other60) { + key = other60.key; + value = other60.value; + __isset = other60.__isset; return *this; } void KeyValue::printTo(std::ostream& out) const { @@ -3544,15 +3656,15 @@ void swap(SortingColumn &a, SortingColumn &b) { swap(a.nulls_first, b.nulls_first); } -SortingColumn::SortingColumn(const SortingColumn& other59) { - column_idx = other59.column_idx; - descending = other59.descending; - nulls_first = other59.nulls_first; +SortingColumn::SortingColumn(const SortingColumn& other61) { + column_idx = other61.column_idx; + descending = other61.descending; + nulls_first = other61.nulls_first; } -SortingColumn& SortingColumn::operator=(const SortingColumn& other60) { - column_idx = other60.column_idx; - descending = other60.descending; - nulls_first = other60.nulls_first; +SortingColumn& SortingColumn::operator=(const SortingColumn& other62) { + column_idx = other62.column_idx; + descending = other62.descending; + nulls_first = other62.nulls_first; return *this; } void SortingColumn::printTo(std::ostream& out) const { @@ -3613,9 +3725,9 @@ uint32_t PageEncodingStats::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast61; - xfer += iprot->readI32(ecast61); - this->page_type = (PageType::type)ecast61; + int32_t ecast63; + xfer += iprot->readI32(ecast63); + this->page_type = (PageType::type)ecast63; isset_page_type = true; } else { xfer += iprot->skip(ftype); @@ -3623,9 +3735,9 @@ uint32_t PageEncodingStats::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast62; - xfer += iprot->readI32(ecast62); - this->encoding = (Encoding::type)ecast62; + int32_t ecast64; + xfer += iprot->readI32(ecast64); + this->encoding = (Encoding::type)ecast64; isset_encoding = true; } else { xfer += iprot->skip(ftype); @@ -3686,15 +3798,15 @@ void swap(PageEncodingStats &a, PageEncodingStats &b) { swap(a.count, b.count); } -PageEncodingStats::PageEncodingStats(const PageEncodingStats& other63) { - page_type = other63.page_type; - encoding = other63.encoding; - count = other63.count; +PageEncodingStats::PageEncodingStats(const PageEncodingStats& other65) { + page_type = other65.page_type; + encoding = other65.encoding; + count = other65.count; } -PageEncodingStats& PageEncodingStats::operator=(const PageEncodingStats& other64) { - page_type = other64.page_type; - encoding = other64.encoding; - count = other64.count; +PageEncodingStats& PageEncodingStats::operator=(const PageEncodingStats& other66) { + page_type = other66.page_type; + encoding = other66.encoding; + count = other66.count; return *this; } void PageEncodingStats::printTo(std::ostream& out) const { @@ -3805,9 +3917,9 @@ uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast65; - xfer += iprot->readI32(ecast65); - this->type = (Type::type)ecast65; + int32_t ecast67; + xfer += iprot->readI32(ecast67); + this->type = (Type::type)ecast67; isset_type = true; } else { xfer += iprot->skip(ftype); @@ -3817,16 +3929,16 @@ uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->encodings.clear(); - uint32_t _size66; - ::apache::thrift::protocol::TType _etype69; - xfer += iprot->readListBegin(_etype69, _size66); - this->encodings.resize(_size66); - uint32_t _i70; - for (_i70 = 0; _i70 < _size66; ++_i70) + uint32_t _size68; + ::apache::thrift::protocol::TType _etype71; + xfer += iprot->readListBegin(_etype71, _size68); + this->encodings.resize(_size68); + uint32_t _i72; + for (_i72 = 0; _i72 < _size68; ++_i72) { - int32_t ecast71; - xfer += iprot->readI32(ecast71); - this->encodings[_i70] = (Encoding::type)ecast71; + int32_t ecast73; + xfer += iprot->readI32(ecast73); + this->encodings[_i72] = (Encoding::type)ecast73; } xfer += iprot->readListEnd(); } @@ -3839,14 +3951,14 @@ uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->path_in_schema.clear(); - uint32_t _size72; - ::apache::thrift::protocol::TType _etype75; - xfer += iprot->readListBegin(_etype75, _size72); - this->path_in_schema.resize(_size72); - uint32_t _i76; - for (_i76 = 0; _i76 < _size72; ++_i76) + uint32_t _size74; + ::apache::thrift::protocol::TType _etype77; + xfer += iprot->readListBegin(_etype77, _size74); + this->path_in_schema.resize(_size74); + uint32_t _i78; + for (_i78 = 0; _i78 < _size74; ++_i78) { - xfer += iprot->readString(this->path_in_schema[_i76]); + xfer += iprot->readString(this->path_in_schema[_i78]); } xfer += iprot->readListEnd(); } @@ -3857,9 +3969,9 @@ uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast77; - xfer += iprot->readI32(ecast77); - this->codec = (CompressionCodec::type)ecast77; + int32_t ecast79; + xfer += iprot->readI32(ecast79); + this->codec = (CompressionCodec::type)ecast79; isset_codec = true; } else { xfer += iprot->skip(ftype); @@ -3893,14 +4005,14 @@ uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->key_value_metadata.clear(); - uint32_t _size78; - ::apache::thrift::protocol::TType _etype81; - xfer += iprot->readListBegin(_etype81, _size78); - this->key_value_metadata.resize(_size78); - uint32_t _i82; - for (_i82 = 0; _i82 < _size78; ++_i82) + uint32_t _size80; + ::apache::thrift::protocol::TType _etype83; + xfer += iprot->readListBegin(_etype83, _size80); + this->key_value_metadata.resize(_size80); + uint32_t _i84; + for (_i84 = 0; _i84 < _size80; ++_i84) { - xfer += this->key_value_metadata[_i82].read(iprot); + xfer += this->key_value_metadata[_i84].read(iprot); } xfer += iprot->readListEnd(); } @@ -3945,14 +4057,14 @@ uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->encoding_stats.clear(); - uint32_t _size83; - ::apache::thrift::protocol::TType _etype86; - xfer += iprot->readListBegin(_etype86, _size83); - this->encoding_stats.resize(_size83); - uint32_t _i87; - for (_i87 = 0; _i87 < _size83; ++_i87) + uint32_t _size85; + ::apache::thrift::protocol::TType _etype88; + xfer += iprot->readListBegin(_etype88, _size85); + this->encoding_stats.resize(_size85); + uint32_t _i89; + for (_i89 = 0; _i89 < _size85; ++_i89) { - xfer += this->encoding_stats[_i87].read(iprot); + xfer += this->encoding_stats[_i89].read(iprot); } xfer += iprot->readListEnd(); } @@ -4001,10 +4113,10 @@ uint32_t ColumnMetaData::write(::apache::thrift::protocol::TProtocol* oprot) con xfer += oprot->writeFieldBegin("encodings", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->encodings.size())); - std::vector ::const_iterator _iter88; - for (_iter88 = this->encodings.begin(); _iter88 != this->encodings.end(); ++_iter88) + std::vector ::const_iterator _iter90; + for (_iter90 = this->encodings.begin(); _iter90 != this->encodings.end(); ++_iter90) { - xfer += oprot->writeI32((int32_t)(*_iter88)); + xfer += oprot->writeI32((int32_t)(*_iter90)); } xfer += oprot->writeListEnd(); } @@ -4013,10 +4125,10 @@ uint32_t ColumnMetaData::write(::apache::thrift::protocol::TProtocol* oprot) con xfer += oprot->writeFieldBegin("path_in_schema", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->path_in_schema.size())); - std::vector ::const_iterator _iter89; - for (_iter89 = this->path_in_schema.begin(); _iter89 != this->path_in_schema.end(); ++_iter89) + std::vector ::const_iterator _iter91; + for (_iter91 = this->path_in_schema.begin(); _iter91 != this->path_in_schema.end(); ++_iter91) { - xfer += oprot->writeString((*_iter89)); + xfer += oprot->writeString((*_iter91)); } xfer += oprot->writeListEnd(); } @@ -4042,10 +4154,10 @@ uint32_t ColumnMetaData::write(::apache::thrift::protocol::TProtocol* oprot) con xfer += oprot->writeFieldBegin("key_value_metadata", ::apache::thrift::protocol::T_LIST, 8); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->key_value_metadata.size())); - std::vector ::const_iterator _iter90; - for (_iter90 = this->key_value_metadata.begin(); _iter90 != this->key_value_metadata.end(); ++_iter90) + std::vector ::const_iterator _iter92; + for (_iter92 = this->key_value_metadata.begin(); _iter92 != this->key_value_metadata.end(); ++_iter92) { - xfer += (*_iter90).write(oprot); + xfer += (*_iter92).write(oprot); } xfer += oprot->writeListEnd(); } @@ -4074,10 +4186,10 @@ uint32_t ColumnMetaData::write(::apache::thrift::protocol::TProtocol* oprot) con xfer += oprot->writeFieldBegin("encoding_stats", ::apache::thrift::protocol::T_LIST, 13); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->encoding_stats.size())); - std::vector ::const_iterator _iter91; - for (_iter91 = this->encoding_stats.begin(); _iter91 != this->encoding_stats.end(); ++_iter91) + std::vector ::const_iterator _iter93; + for (_iter93 = this->encoding_stats.begin(); _iter93 != this->encoding_stats.end(); ++_iter93) { - xfer += (*_iter91).write(oprot); + xfer += (*_iter93).write(oprot); } xfer += oprot->writeListEnd(); } @@ -4106,37 +4218,37 @@ void swap(ColumnMetaData &a, ColumnMetaData &b) { swap(a.__isset, b.__isset); } -ColumnMetaData::ColumnMetaData(const ColumnMetaData& other92) { - type = other92.type; - encodings = other92.encodings; - path_in_schema = other92.path_in_schema; - codec = other92.codec; - num_values = other92.num_values; - total_uncompressed_size = other92.total_uncompressed_size; - total_compressed_size = other92.total_compressed_size; - key_value_metadata = other92.key_value_metadata; - data_page_offset = other92.data_page_offset; - index_page_offset = other92.index_page_offset; - dictionary_page_offset = other92.dictionary_page_offset; - statistics = other92.statistics; - encoding_stats = other92.encoding_stats; - __isset = other92.__isset; +ColumnMetaData::ColumnMetaData(const ColumnMetaData& other94) { + type = other94.type; + encodings = other94.encodings; + path_in_schema = other94.path_in_schema; + codec = other94.codec; + num_values = other94.num_values; + total_uncompressed_size = other94.total_uncompressed_size; + total_compressed_size = other94.total_compressed_size; + key_value_metadata = other94.key_value_metadata; + data_page_offset = other94.data_page_offset; + index_page_offset = other94.index_page_offset; + dictionary_page_offset = other94.dictionary_page_offset; + statistics = other94.statistics; + encoding_stats = other94.encoding_stats; + __isset = other94.__isset; } -ColumnMetaData& ColumnMetaData::operator=(const ColumnMetaData& other93) { - type = other93.type; - encodings = other93.encodings; - path_in_schema = other93.path_in_schema; - codec = other93.codec; - num_values = other93.num_values; - total_uncompressed_size = other93.total_uncompressed_size; - total_compressed_size = other93.total_compressed_size; - key_value_metadata = other93.key_value_metadata; - data_page_offset = other93.data_page_offset; - index_page_offset = other93.index_page_offset; - dictionary_page_offset = other93.dictionary_page_offset; - statistics = other93.statistics; - encoding_stats = other93.encoding_stats; - __isset = other93.__isset; +ColumnMetaData& ColumnMetaData::operator=(const ColumnMetaData& other95) { + type = other95.type; + encodings = other95.encodings; + path_in_schema = other95.path_in_schema; + codec = other95.codec; + num_values = other95.num_values; + total_uncompressed_size = other95.total_uncompressed_size; + total_compressed_size = other95.total_compressed_size; + key_value_metadata = other95.key_value_metadata; + data_page_offset = other95.data_page_offset; + index_page_offset = other95.index_page_offset; + dictionary_page_offset = other95.dictionary_page_offset; + statistics = other95.statistics; + encoding_stats = other95.encoding_stats; + __isset = other95.__isset; return *this; } void ColumnMetaData::printTo(std::ostream& out) const { @@ -4213,11 +4325,11 @@ void swap(EncryptionWithFooterKey &a, EncryptionWithFooterKey &b) { (void) b; } -EncryptionWithFooterKey::EncryptionWithFooterKey(const EncryptionWithFooterKey& other94) { - (void) other94; +EncryptionWithFooterKey::EncryptionWithFooterKey(const EncryptionWithFooterKey& other96) { + (void) other96; } -EncryptionWithFooterKey& EncryptionWithFooterKey::operator=(const EncryptionWithFooterKey& other95) { - (void) other95; +EncryptionWithFooterKey& EncryptionWithFooterKey::operator=(const EncryptionWithFooterKey& other97) { + (void) other97; return *this; } void EncryptionWithFooterKey::printTo(std::ostream& out) const { @@ -4235,9 +4347,9 @@ void EncryptionWithColumnKey::__set_path_in_schema(const std::vectorpath_in_schema = val; } -void EncryptionWithColumnKey::__set_column_key_metadata(const std::string& val) { - this->column_key_metadata = val; -__isset.column_key_metadata = true; +void EncryptionWithColumnKey::__set_key_metadata(const std::string& val) { + this->key_metadata = val; +__isset.key_metadata = true; } std::ostream& operator<<(std::ostream& out, const EncryptionWithColumnKey& obj) { @@ -4272,14 +4384,14 @@ uint32_t EncryptionWithColumnKey::read(::apache::thrift::protocol::TProtocol* ip if (ftype == ::apache::thrift::protocol::T_LIST) { { this->path_in_schema.clear(); - uint32_t _size96; - ::apache::thrift::protocol::TType _etype99; - xfer += iprot->readListBegin(_etype99, _size96); - this->path_in_schema.resize(_size96); - uint32_t _i100; - for (_i100 = 0; _i100 < _size96; ++_i100) + uint32_t _size98; + ::apache::thrift::protocol::TType _etype101; + xfer += iprot->readListBegin(_etype101, _size98); + this->path_in_schema.resize(_size98); + uint32_t _i102; + for (_i102 = 0; _i102 < _size98; ++_i102) { - xfer += iprot->readString(this->path_in_schema[_i100]); + xfer += iprot->readString(this->path_in_schema[_i102]); } xfer += iprot->readListEnd(); } @@ -4290,8 +4402,8 @@ uint32_t EncryptionWithColumnKey::read(::apache::thrift::protocol::TProtocol* ip break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->column_key_metadata); - this->__isset.column_key_metadata = true; + xfer += iprot->readBinary(this->key_metadata); + this->__isset.key_metadata = true; } else { xfer += iprot->skip(ftype); } @@ -4318,18 +4430,18 @@ uint32_t EncryptionWithColumnKey::write(::apache::thrift::protocol::TProtocol* o xfer += oprot->writeFieldBegin("path_in_schema", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->path_in_schema.size())); - std::vector ::const_iterator _iter101; - for (_iter101 = this->path_in_schema.begin(); _iter101 != this->path_in_schema.end(); ++_iter101) + std::vector ::const_iterator _iter103; + for (_iter103 = this->path_in_schema.begin(); _iter103 != this->path_in_schema.end(); ++_iter103) { - xfer += oprot->writeString((*_iter101)); + xfer += oprot->writeString((*_iter103)); } xfer += oprot->writeListEnd(); } xfer += oprot->writeFieldEnd(); - if (this->__isset.column_key_metadata) { - xfer += oprot->writeFieldBegin("column_key_metadata", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->column_key_metadata); + if (this->__isset.key_metadata) { + xfer += oprot->writeFieldBegin("key_metadata", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->key_metadata); xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); @@ -4340,26 +4452,26 @@ uint32_t EncryptionWithColumnKey::write(::apache::thrift::protocol::TProtocol* o void swap(EncryptionWithColumnKey &a, EncryptionWithColumnKey &b) { using ::std::swap; swap(a.path_in_schema, b.path_in_schema); - swap(a.column_key_metadata, b.column_key_metadata); + swap(a.key_metadata, b.key_metadata); swap(a.__isset, b.__isset); } -EncryptionWithColumnKey::EncryptionWithColumnKey(const EncryptionWithColumnKey& other102) { - path_in_schema = other102.path_in_schema; - column_key_metadata = other102.column_key_metadata; - __isset = other102.__isset; +EncryptionWithColumnKey::EncryptionWithColumnKey(const EncryptionWithColumnKey& other104) { + path_in_schema = other104.path_in_schema; + key_metadata = other104.key_metadata; + __isset = other104.__isset; } -EncryptionWithColumnKey& EncryptionWithColumnKey::operator=(const EncryptionWithColumnKey& other103) { - path_in_schema = other103.path_in_schema; - column_key_metadata = other103.column_key_metadata; - __isset = other103.__isset; +EncryptionWithColumnKey& EncryptionWithColumnKey::operator=(const EncryptionWithColumnKey& other105) { + path_in_schema = other105.path_in_schema; + key_metadata = other105.key_metadata; + __isset = other105.__isset; return *this; } void EncryptionWithColumnKey::printTo(std::ostream& out) const { using ::apache::thrift::to_string; out << "EncryptionWithColumnKey("; out << "path_in_schema=" << to_string(path_in_schema); - out << ", " << "column_key_metadata="; (__isset.column_key_metadata ? (out << to_string(column_key_metadata)) : (out << "")); + out << ", " << "key_metadata="; (__isset.key_metadata ? (out << to_string(key_metadata)) : (out << "")); out << ")"; } @@ -4460,15 +4572,15 @@ void swap(ColumnCryptoMetaData &a, ColumnCryptoMetaData &b) { swap(a.__isset, b.__isset); } -ColumnCryptoMetaData::ColumnCryptoMetaData(const ColumnCryptoMetaData& other104) { - ENCRYPTION_WITH_FOOTER_KEY = other104.ENCRYPTION_WITH_FOOTER_KEY; - ENCRYPTION_WITH_COLUMN_KEY = other104.ENCRYPTION_WITH_COLUMN_KEY; - __isset = other104.__isset; +ColumnCryptoMetaData::ColumnCryptoMetaData(const ColumnCryptoMetaData& other106) { + ENCRYPTION_WITH_FOOTER_KEY = other106.ENCRYPTION_WITH_FOOTER_KEY; + ENCRYPTION_WITH_COLUMN_KEY = other106.ENCRYPTION_WITH_COLUMN_KEY; + __isset = other106.__isset; } -ColumnCryptoMetaData& ColumnCryptoMetaData::operator=(const ColumnCryptoMetaData& other105) { - ENCRYPTION_WITH_FOOTER_KEY = other105.ENCRYPTION_WITH_FOOTER_KEY; - ENCRYPTION_WITH_COLUMN_KEY = other105.ENCRYPTION_WITH_COLUMN_KEY; - __isset = other105.__isset; +ColumnCryptoMetaData& ColumnCryptoMetaData::operator=(const ColumnCryptoMetaData& other107) { + ENCRYPTION_WITH_FOOTER_KEY = other107.ENCRYPTION_WITH_FOOTER_KEY; + ENCRYPTION_WITH_COLUMN_KEY = other107.ENCRYPTION_WITH_COLUMN_KEY; + __isset = other107.__isset; return *this; } void ColumnCryptoMetaData::printTo(std::ostream& out) const { @@ -4518,9 +4630,14 @@ void ColumnChunk::__set_column_index_length(const int32_t val) { __isset.column_index_length = true; } -void ColumnChunk::__set_crypto_meta_data(const ColumnCryptoMetaData& val) { - this->crypto_meta_data = val; -__isset.crypto_meta_data = true; +void ColumnChunk::__set_crypto_metadata(const ColumnCryptoMetaData& val) { + this->crypto_metadata = val; +__isset.crypto_metadata = true; +} + +void ColumnChunk::__set_encrypted_column_metadata(const std::string& val) { + this->encrypted_column_metadata = val; +__isset.encrypted_column_metadata = true; } std::ostream& operator<<(std::ostream& out, const ColumnChunk& obj) { @@ -4609,8 +4726,16 @@ uint32_t ColumnChunk::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 8: if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->crypto_meta_data.read(iprot); - this->__isset.crypto_meta_data = true; + xfer += this->crypto_metadata.read(iprot); + this->__isset.crypto_metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->encrypted_column_metadata); + this->__isset.encrypted_column_metadata = true; } else { xfer += iprot->skip(ftype); } @@ -4668,9 +4793,14 @@ uint32_t ColumnChunk::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += oprot->writeI32(this->column_index_length); xfer += oprot->writeFieldEnd(); } - if (this->__isset.crypto_meta_data) { - xfer += oprot->writeFieldBegin("crypto_meta_data", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->crypto_meta_data.write(oprot); + if (this->__isset.crypto_metadata) { + xfer += oprot->writeFieldBegin("crypto_metadata", ::apache::thrift::protocol::T_STRUCT, 8); + xfer += this->crypto_metadata.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.encrypted_column_metadata) { + xfer += oprot->writeFieldBegin("encrypted_column_metadata", ::apache::thrift::protocol::T_STRING, 9); + xfer += oprot->writeBinary(this->encrypted_column_metadata); xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); @@ -4687,31 +4817,34 @@ void swap(ColumnChunk &a, ColumnChunk &b) { swap(a.offset_index_length, b.offset_index_length); swap(a.column_index_offset, b.column_index_offset); swap(a.column_index_length, b.column_index_length); - swap(a.crypto_meta_data, b.crypto_meta_data); + swap(a.crypto_metadata, b.crypto_metadata); + swap(a.encrypted_column_metadata, b.encrypted_column_metadata); swap(a.__isset, b.__isset); } -ColumnChunk::ColumnChunk(const ColumnChunk& other106) { - file_path = other106.file_path; - file_offset = other106.file_offset; - meta_data = other106.meta_data; - offset_index_offset = other106.offset_index_offset; - offset_index_length = other106.offset_index_length; - column_index_offset = other106.column_index_offset; - column_index_length = other106.column_index_length; - crypto_meta_data = other106.crypto_meta_data; - __isset = other106.__isset; +ColumnChunk::ColumnChunk(const ColumnChunk& other108) { + file_path = other108.file_path; + file_offset = other108.file_offset; + meta_data = other108.meta_data; + offset_index_offset = other108.offset_index_offset; + offset_index_length = other108.offset_index_length; + column_index_offset = other108.column_index_offset; + column_index_length = other108.column_index_length; + crypto_metadata = other108.crypto_metadata; + encrypted_column_metadata = other108.encrypted_column_metadata; + __isset = other108.__isset; } -ColumnChunk& ColumnChunk::operator=(const ColumnChunk& other107) { - file_path = other107.file_path; - file_offset = other107.file_offset; - meta_data = other107.meta_data; - offset_index_offset = other107.offset_index_offset; - offset_index_length = other107.offset_index_length; - column_index_offset = other107.column_index_offset; - column_index_length = other107.column_index_length; - crypto_meta_data = other107.crypto_meta_data; - __isset = other107.__isset; +ColumnChunk& ColumnChunk::operator=(const ColumnChunk& other109) { + file_path = other109.file_path; + file_offset = other109.file_offset; + meta_data = other109.meta_data; + offset_index_offset = other109.offset_index_offset; + offset_index_length = other109.offset_index_length; + column_index_offset = other109.column_index_offset; + column_index_length = other109.column_index_length; + crypto_metadata = other109.crypto_metadata; + encrypted_column_metadata = other109.encrypted_column_metadata; + __isset = other109.__isset; return *this; } void ColumnChunk::printTo(std::ostream& out) const { @@ -4724,7 +4857,8 @@ void ColumnChunk::printTo(std::ostream& out) const { out << ", " << "offset_index_length="; (__isset.offset_index_length ? (out << to_string(offset_index_length)) : (out << "")); out << ", " << "column_index_offset="; (__isset.column_index_offset ? (out << to_string(column_index_offset)) : (out << "")); out << ", " << "column_index_length="; (__isset.column_index_length ? (out << to_string(column_index_length)) : (out << "")); - out << ", " << "crypto_meta_data="; (__isset.crypto_meta_data ? (out << to_string(crypto_meta_data)) : (out << "")); + out << ", " << "crypto_metadata="; (__isset.crypto_metadata ? (out << to_string(crypto_metadata)) : (out << "")); + out << ", " << "encrypted_column_metadata="; (__isset.encrypted_column_metadata ? (out << to_string(encrypted_column_metadata)) : (out << "")); out << ")"; } @@ -4759,6 +4893,11 @@ void RowGroup::__set_total_compressed_size(const int64_t val) { this->total_compressed_size = val; __isset.total_compressed_size = true; } + +void RowGroup::__set_ordinal(const int16_t val) { + this->ordinal = val; +__isset.ordinal = true; +} std::ostream& operator<<(std::ostream& out, const RowGroup& obj) { obj.printTo(out); @@ -4794,14 +4933,14 @@ uint32_t RowGroup::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->columns.clear(); - uint32_t _size108; - ::apache::thrift::protocol::TType _etype111; - xfer += iprot->readListBegin(_etype111, _size108); - this->columns.resize(_size108); - uint32_t _i112; - for (_i112 = 0; _i112 < _size108; ++_i112) + uint32_t _size110; + ::apache::thrift::protocol::TType _etype113; + xfer += iprot->readListBegin(_etype113, _size110); + this->columns.resize(_size110); + uint32_t _i114; + for (_i114 = 0; _i114 < _size110; ++_i114) { - xfer += this->columns[_i112].read(iprot); + xfer += this->columns[_i114].read(iprot); } xfer += iprot->readListEnd(); } @@ -4830,14 +4969,14 @@ uint32_t RowGroup::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->sorting_columns.clear(); - uint32_t _size113; - ::apache::thrift::protocol::TType _etype116; - xfer += iprot->readListBegin(_etype116, _size113); - this->sorting_columns.resize(_size113); - uint32_t _i117; - for (_i117 = 0; _i117 < _size113; ++_i117) + uint32_t _size115; + ::apache::thrift::protocol::TType _etype118; + xfer += iprot->readListBegin(_etype118, _size115); + this->sorting_columns.resize(_size115); + uint32_t _i119; + for (_i119 = 0; _i119 < _size115; ++_i119) { - xfer += this->sorting_columns[_i117].read(iprot); + xfer += this->sorting_columns[_i119].read(iprot); } xfer += iprot->readListEnd(); } @@ -4862,6 +5001,14 @@ uint32_t RowGroup::read(::apache::thrift::protocol::TProtocol* iprot) { xfer += iprot->skip(ftype); } break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->ordinal); + this->__isset.ordinal = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -4888,10 +5035,10 @@ uint32_t RowGroup::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += oprot->writeFieldBegin("columns", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->columns.size())); - std::vector ::const_iterator _iter118; - for (_iter118 = this->columns.begin(); _iter118 != this->columns.end(); ++_iter118) + std::vector ::const_iterator _iter120; + for (_iter120 = this->columns.begin(); _iter120 != this->columns.end(); ++_iter120) { - xfer += (*_iter118).write(oprot); + xfer += (*_iter120).write(oprot); } xfer += oprot->writeListEnd(); } @@ -4909,10 +5056,10 @@ uint32_t RowGroup::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += oprot->writeFieldBegin("sorting_columns", ::apache::thrift::protocol::T_LIST, 4); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->sorting_columns.size())); - std::vector ::const_iterator _iter119; - for (_iter119 = this->sorting_columns.begin(); _iter119 != this->sorting_columns.end(); ++_iter119) + std::vector ::const_iterator _iter121; + for (_iter121 = this->sorting_columns.begin(); _iter121 != this->sorting_columns.end(); ++_iter121) { - xfer += (*_iter119).write(oprot); + xfer += (*_iter121).write(oprot); } xfer += oprot->writeListEnd(); } @@ -4928,6 +5075,11 @@ uint32_t RowGroup::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += oprot->writeI64(this->total_compressed_size); xfer += oprot->writeFieldEnd(); } + if (this->__isset.ordinal) { + xfer += oprot->writeFieldBegin("ordinal", ::apache::thrift::protocol::T_I16, 7); + xfer += oprot->writeI16(this->ordinal); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -4941,26 +5093,29 @@ void swap(RowGroup &a, RowGroup &b) { swap(a.sorting_columns, b.sorting_columns); swap(a.file_offset, b.file_offset); swap(a.total_compressed_size, b.total_compressed_size); + swap(a.ordinal, b.ordinal); swap(a.__isset, b.__isset); } -RowGroup::RowGroup(const RowGroup& other120) { - columns = other120.columns; - total_byte_size = other120.total_byte_size; - num_rows = other120.num_rows; - sorting_columns = other120.sorting_columns; - file_offset = other120.file_offset; - total_compressed_size = other120.total_compressed_size; - __isset = other120.__isset; +RowGroup::RowGroup(const RowGroup& other122) { + columns = other122.columns; + total_byte_size = other122.total_byte_size; + num_rows = other122.num_rows; + sorting_columns = other122.sorting_columns; + file_offset = other122.file_offset; + total_compressed_size = other122.total_compressed_size; + ordinal = other122.ordinal; + __isset = other122.__isset; } -RowGroup& RowGroup::operator=(const RowGroup& other121) { - columns = other121.columns; - total_byte_size = other121.total_byte_size; - num_rows = other121.num_rows; - sorting_columns = other121.sorting_columns; - file_offset = other121.file_offset; - total_compressed_size = other121.total_compressed_size; - __isset = other121.__isset; +RowGroup& RowGroup::operator=(const RowGroup& other123) { + columns = other123.columns; + total_byte_size = other123.total_byte_size; + num_rows = other123.num_rows; + sorting_columns = other123.sorting_columns; + file_offset = other123.file_offset; + total_compressed_size = other123.total_compressed_size; + ordinal = other123.ordinal; + __isset = other123.__isset; return *this; } void RowGroup::printTo(std::ostream& out) const { @@ -4972,6 +5127,7 @@ void RowGroup::printTo(std::ostream& out) const { out << ", " << "sorting_columns="; (__isset.sorting_columns ? (out << to_string(sorting_columns)) : (out << "")); out << ", " << "file_offset="; (__isset.file_offset ? (out << to_string(file_offset)) : (out << "")); out << ", " << "total_compressed_size="; (__isset.total_compressed_size ? (out << to_string(total_compressed_size)) : (out << "")); + out << ", " << "ordinal="; (__isset.ordinal ? (out << to_string(ordinal)) : (out << "")); out << ")"; } @@ -5030,11 +5186,11 @@ void swap(TypeDefinedOrder &a, TypeDefinedOrder &b) { (void) b; } -TypeDefinedOrder::TypeDefinedOrder(const TypeDefinedOrder& other122) { - (void) other122; +TypeDefinedOrder::TypeDefinedOrder(const TypeDefinedOrder& other124) { + (void) other124; } -TypeDefinedOrder& TypeDefinedOrder::operator=(const TypeDefinedOrder& other123) { - (void) other123; +TypeDefinedOrder& TypeDefinedOrder::operator=(const TypeDefinedOrder& other125) { + (void) other125; return *this; } void TypeDefinedOrder::printTo(std::ostream& out) const { @@ -5121,13 +5277,13 @@ void swap(ColumnOrder &a, ColumnOrder &b) { swap(a.__isset, b.__isset); } -ColumnOrder::ColumnOrder(const ColumnOrder& other124) { - TYPE_ORDER = other124.TYPE_ORDER; - __isset = other124.__isset; +ColumnOrder::ColumnOrder(const ColumnOrder& other126) { + TYPE_ORDER = other126.TYPE_ORDER; + __isset = other126.__isset; } -ColumnOrder& ColumnOrder::operator=(const ColumnOrder& other125) { - TYPE_ORDER = other125.TYPE_ORDER; - __isset = other125.__isset; +ColumnOrder& ColumnOrder::operator=(const ColumnOrder& other127) { + TYPE_ORDER = other127.TYPE_ORDER; + __isset = other127.__isset; return *this; } void ColumnOrder::printTo(std::ostream& out) const { @@ -5255,15 +5411,15 @@ void swap(PageLocation &a, PageLocation &b) { swap(a.first_row_index, b.first_row_index); } -PageLocation::PageLocation(const PageLocation& other126) { - offset = other126.offset; - compressed_page_size = other126.compressed_page_size; - first_row_index = other126.first_row_index; +PageLocation::PageLocation(const PageLocation& other128) { + offset = other128.offset; + compressed_page_size = other128.compressed_page_size; + first_row_index = other128.first_row_index; } -PageLocation& PageLocation::operator=(const PageLocation& other127) { - offset = other127.offset; - compressed_page_size = other127.compressed_page_size; - first_row_index = other127.first_row_index; +PageLocation& PageLocation::operator=(const PageLocation& other129) { + offset = other129.offset; + compressed_page_size = other129.compressed_page_size; + first_row_index = other129.first_row_index; return *this; } void PageLocation::printTo(std::ostream& out) const { @@ -5316,14 +5472,14 @@ uint32_t OffsetIndex::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->page_locations.clear(); - uint32_t _size128; - ::apache::thrift::protocol::TType _etype131; - xfer += iprot->readListBegin(_etype131, _size128); - this->page_locations.resize(_size128); - uint32_t _i132; - for (_i132 = 0; _i132 < _size128; ++_i132) + uint32_t _size130; + ::apache::thrift::protocol::TType _etype133; + xfer += iprot->readListBegin(_etype133, _size130); + this->page_locations.resize(_size130); + uint32_t _i134; + for (_i134 = 0; _i134 < _size130; ++_i134) { - xfer += this->page_locations[_i132].read(iprot); + xfer += this->page_locations[_i134].read(iprot); } xfer += iprot->readListEnd(); } @@ -5354,10 +5510,10 @@ uint32_t OffsetIndex::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += oprot->writeFieldBegin("page_locations", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->page_locations.size())); - std::vector ::const_iterator _iter133; - for (_iter133 = this->page_locations.begin(); _iter133 != this->page_locations.end(); ++_iter133) + std::vector ::const_iterator _iter135; + for (_iter135 = this->page_locations.begin(); _iter135 != this->page_locations.end(); ++_iter135) { - xfer += (*_iter133).write(oprot); + xfer += (*_iter135).write(oprot); } xfer += oprot->writeListEnd(); } @@ -5373,11 +5529,11 @@ void swap(OffsetIndex &a, OffsetIndex &b) { swap(a.page_locations, b.page_locations); } -OffsetIndex::OffsetIndex(const OffsetIndex& other134) { - page_locations = other134.page_locations; +OffsetIndex::OffsetIndex(const OffsetIndex& other136) { + page_locations = other136.page_locations; } -OffsetIndex& OffsetIndex::operator=(const OffsetIndex& other135) { - page_locations = other135.page_locations; +OffsetIndex& OffsetIndex::operator=(const OffsetIndex& other137) { + page_locations = other137.page_locations; return *this; } void OffsetIndex::printTo(std::ostream& out) const { @@ -5448,14 +5604,14 @@ uint32_t ColumnIndex::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->null_pages.clear(); - uint32_t _size136; - ::apache::thrift::protocol::TType _etype139; - xfer += iprot->readListBegin(_etype139, _size136); - this->null_pages.resize(_size136); - uint32_t _i140; - for (_i140 = 0; _i140 < _size136; ++_i140) + uint32_t _size138; + ::apache::thrift::protocol::TType _etype141; + xfer += iprot->readListBegin(_etype141, _size138); + this->null_pages.resize(_size138); + uint32_t _i142; + for (_i142 = 0; _i142 < _size138; ++_i142) { - xfer += iprot->readBool(this->null_pages[_i140]); + xfer += iprot->readBool(this->null_pages[_i142]); } xfer += iprot->readListEnd(); } @@ -5468,14 +5624,14 @@ uint32_t ColumnIndex::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->min_values.clear(); - uint32_t _size141; - ::apache::thrift::protocol::TType _etype144; - xfer += iprot->readListBegin(_etype144, _size141); - this->min_values.resize(_size141); - uint32_t _i145; - for (_i145 = 0; _i145 < _size141; ++_i145) + uint32_t _size143; + ::apache::thrift::protocol::TType _etype146; + xfer += iprot->readListBegin(_etype146, _size143); + this->min_values.resize(_size143); + uint32_t _i147; + for (_i147 = 0; _i147 < _size143; ++_i147) { - xfer += iprot->readBinary(this->min_values[_i145]); + xfer += iprot->readBinary(this->min_values[_i147]); } xfer += iprot->readListEnd(); } @@ -5488,14 +5644,14 @@ uint32_t ColumnIndex::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->max_values.clear(); - uint32_t _size146; - ::apache::thrift::protocol::TType _etype149; - xfer += iprot->readListBegin(_etype149, _size146); - this->max_values.resize(_size146); - uint32_t _i150; - for (_i150 = 0; _i150 < _size146; ++_i150) + uint32_t _size148; + ::apache::thrift::protocol::TType _etype151; + xfer += iprot->readListBegin(_etype151, _size148); + this->max_values.resize(_size148); + uint32_t _i152; + for (_i152 = 0; _i152 < _size148; ++_i152) { - xfer += iprot->readBinary(this->max_values[_i150]); + xfer += iprot->readBinary(this->max_values[_i152]); } xfer += iprot->readListEnd(); } @@ -5506,9 +5662,9 @@ uint32_t ColumnIndex::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast151; - xfer += iprot->readI32(ecast151); - this->boundary_order = (BoundaryOrder::type)ecast151; + int32_t ecast153; + xfer += iprot->readI32(ecast153); + this->boundary_order = (BoundaryOrder::type)ecast153; isset_boundary_order = true; } else { xfer += iprot->skip(ftype); @@ -5518,14 +5674,14 @@ uint32_t ColumnIndex::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->null_counts.clear(); - uint32_t _size152; - ::apache::thrift::protocol::TType _etype155; - xfer += iprot->readListBegin(_etype155, _size152); - this->null_counts.resize(_size152); - uint32_t _i156; - for (_i156 = 0; _i156 < _size152; ++_i156) + uint32_t _size154; + ::apache::thrift::protocol::TType _etype157; + xfer += iprot->readListBegin(_etype157, _size154); + this->null_counts.resize(_size154); + uint32_t _i158; + for (_i158 = 0; _i158 < _size154; ++_i158) { - xfer += iprot->readI64(this->null_counts[_i156]); + xfer += iprot->readI64(this->null_counts[_i158]); } xfer += iprot->readListEnd(); } @@ -5562,10 +5718,10 @@ uint32_t ColumnIndex::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += oprot->writeFieldBegin("null_pages", ::apache::thrift::protocol::T_LIST, 1); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_BOOL, static_cast(this->null_pages.size())); - std::vector ::const_iterator _iter157; - for (_iter157 = this->null_pages.begin(); _iter157 != this->null_pages.end(); ++_iter157) + std::vector ::const_iterator _iter159; + for (_iter159 = this->null_pages.begin(); _iter159 != this->null_pages.end(); ++_iter159) { - xfer += oprot->writeBool((*_iter157)); + xfer += oprot->writeBool((*_iter159)); } xfer += oprot->writeListEnd(); } @@ -5574,10 +5730,10 @@ uint32_t ColumnIndex::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += oprot->writeFieldBegin("min_values", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->min_values.size())); - std::vector ::const_iterator _iter158; - for (_iter158 = this->min_values.begin(); _iter158 != this->min_values.end(); ++_iter158) + std::vector ::const_iterator _iter160; + for (_iter160 = this->min_values.begin(); _iter160 != this->min_values.end(); ++_iter160) { - xfer += oprot->writeBinary((*_iter158)); + xfer += oprot->writeBinary((*_iter160)); } xfer += oprot->writeListEnd(); } @@ -5586,10 +5742,10 @@ uint32_t ColumnIndex::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += oprot->writeFieldBegin("max_values", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->max_values.size())); - std::vector ::const_iterator _iter159; - for (_iter159 = this->max_values.begin(); _iter159 != this->max_values.end(); ++_iter159) + std::vector ::const_iterator _iter161; + for (_iter161 = this->max_values.begin(); _iter161 != this->max_values.end(); ++_iter161) { - xfer += oprot->writeBinary((*_iter159)); + xfer += oprot->writeBinary((*_iter161)); } xfer += oprot->writeListEnd(); } @@ -5603,10 +5759,10 @@ uint32_t ColumnIndex::write(::apache::thrift::protocol::TProtocol* oprot) const xfer += oprot->writeFieldBegin("null_counts", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->null_counts.size())); - std::vector ::const_iterator _iter160; - for (_iter160 = this->null_counts.begin(); _iter160 != this->null_counts.end(); ++_iter160) + std::vector ::const_iterator _iter162; + for (_iter162 = this->null_counts.begin(); _iter162 != this->null_counts.end(); ++_iter162) { - xfer += oprot->writeI64((*_iter160)); + xfer += oprot->writeI64((*_iter162)); } xfer += oprot->writeListEnd(); } @@ -5627,21 +5783,21 @@ void swap(ColumnIndex &a, ColumnIndex &b) { swap(a.__isset, b.__isset); } -ColumnIndex::ColumnIndex(const ColumnIndex& other161) { - null_pages = other161.null_pages; - min_values = other161.min_values; - max_values = other161.max_values; - boundary_order = other161.boundary_order; - null_counts = other161.null_counts; - __isset = other161.__isset; +ColumnIndex::ColumnIndex(const ColumnIndex& other163) { + null_pages = other163.null_pages; + min_values = other163.min_values; + max_values = other163.max_values; + boundary_order = other163.boundary_order; + null_counts = other163.null_counts; + __isset = other163.__isset; } -ColumnIndex& ColumnIndex::operator=(const ColumnIndex& other162) { - null_pages = other162.null_pages; - min_values = other162.min_values; - max_values = other162.max_values; - boundary_order = other162.boundary_order; - null_counts = other162.null_counts; - __isset = other162.__isset; +ColumnIndex& ColumnIndex::operator=(const ColumnIndex& other164) { + null_pages = other164.null_pages; + min_values = other164.min_values; + max_values = other164.max_values; + boundary_order = other164.boundary_order; + null_counts = other164.null_counts; + __isset = other164.__isset; return *this; } void ColumnIndex::printTo(std::ostream& out) const { @@ -5656,328 +5812,23 @@ void ColumnIndex::printTo(std::ostream& out) const { } -FileMetaData::~FileMetaData() throw() { -} - - -void FileMetaData::__set_version(const int32_t val) { - this->version = val; -} - -void FileMetaData::__set_schema(const std::vector & val) { - this->schema = val; -} - -void FileMetaData::__set_num_rows(const int64_t val) { - this->num_rows = val; -} - -void FileMetaData::__set_row_groups(const std::vector & val) { - this->row_groups = val; -} - -void FileMetaData::__set_key_value_metadata(const std::vector & val) { - this->key_value_metadata = val; -__isset.key_value_metadata = true; -} - -void FileMetaData::__set_created_by(const std::string& val) { - this->created_by = val; -__isset.created_by = true; -} - -void FileMetaData::__set_column_orders(const std::vector & val) { - this->column_orders = val; -__isset.column_orders = true; -} -std::ostream& operator<<(std::ostream& out, const FileMetaData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FileMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_version = false; - bool isset_schema = false; - bool isset_num_rows = false; - bool isset_row_groups = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->version); - isset_version = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->schema.clear(); - uint32_t _size163; - ::apache::thrift::protocol::TType _etype166; - xfer += iprot->readListBegin(_etype166, _size163); - this->schema.resize(_size163); - uint32_t _i167; - for (_i167 = 0; _i167 < _size163; ++_i167) - { - xfer += this->schema[_i167].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_schema = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->num_rows); - isset_num_rows = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->row_groups.clear(); - uint32_t _size168; - ::apache::thrift::protocol::TType _etype171; - xfer += iprot->readListBegin(_etype171, _size168); - this->row_groups.resize(_size168); - uint32_t _i172; - for (_i172 = 0; _i172 < _size168; ++_i172) - { - xfer += this->row_groups[_i172].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_row_groups = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->key_value_metadata.clear(); - uint32_t _size173; - ::apache::thrift::protocol::TType _etype176; - xfer += iprot->readListBegin(_etype176, _size173); - this->key_value_metadata.resize(_size173); - uint32_t _i177; - for (_i177 = 0; _i177 < _size173; ++_i177) - { - xfer += this->key_value_metadata[_i177].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.key_value_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->created_by); - this->__isset.created_by = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->column_orders.clear(); - uint32_t _size178; - ::apache::thrift::protocol::TType _etype181; - xfer += iprot->readListBegin(_etype181, _size178); - this->column_orders.resize(_size178); - uint32_t _i182; - for (_i182 = 0; _i182 < _size178; ++_i182) - { - xfer += this->column_orders[_i182].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.column_orders = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_version) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_schema) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_num_rows) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_row_groups) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t FileMetaData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FileMetaData"); - - xfer += oprot->writeFieldBegin("version", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->version); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("schema", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->schema.size())); - std::vector ::const_iterator _iter183; - for (_iter183 = this->schema.begin(); _iter183 != this->schema.end(); ++_iter183) - { - xfer += (*_iter183).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("num_rows", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->num_rows); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("row_groups", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->row_groups.size())); - std::vector ::const_iterator _iter184; - for (_iter184 = this->row_groups.begin(); _iter184 != this->row_groups.end(); ++_iter184) - { - xfer += (*_iter184).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.key_value_metadata) { - xfer += oprot->writeFieldBegin("key_value_metadata", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->key_value_metadata.size())); - std::vector ::const_iterator _iter185; - for (_iter185 = this->key_value_metadata.begin(); _iter185 != this->key_value_metadata.end(); ++_iter185) - { - xfer += (*_iter185).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.created_by) { - xfer += oprot->writeFieldBegin("created_by", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->created_by); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.column_orders) { - xfer += oprot->writeFieldBegin("column_orders", ::apache::thrift::protocol::T_LIST, 7); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->column_orders.size())); - std::vector ::const_iterator _iter186; - for (_iter186 = this->column_orders.begin(); _iter186 != this->column_orders.end(); ++_iter186) - { - xfer += (*_iter186).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FileMetaData &a, FileMetaData &b) { - using ::std::swap; - swap(a.version, b.version); - swap(a.schema, b.schema); - swap(a.num_rows, b.num_rows); - swap(a.row_groups, b.row_groups); - swap(a.key_value_metadata, b.key_value_metadata); - swap(a.created_by, b.created_by); - swap(a.column_orders, b.column_orders); - swap(a.__isset, b.__isset); -} - -FileMetaData::FileMetaData(const FileMetaData& other187) { - version = other187.version; - schema = other187.schema; - num_rows = other187.num_rows; - row_groups = other187.row_groups; - key_value_metadata = other187.key_value_metadata; - created_by = other187.created_by; - column_orders = other187.column_orders; - __isset = other187.__isset; -} -FileMetaData& FileMetaData::operator=(const FileMetaData& other188) { - version = other188.version; - schema = other188.schema; - num_rows = other188.num_rows; - row_groups = other188.row_groups; - key_value_metadata = other188.key_value_metadata; - created_by = other188.created_by; - column_orders = other188.column_orders; - __isset = other188.__isset; - return *this; -} -void FileMetaData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FileMetaData("; - out << "version=" << to_string(version); - out << ", " << "schema=" << to_string(schema); - out << ", " << "num_rows=" << to_string(num_rows); - out << ", " << "row_groups=" << to_string(row_groups); - out << ", " << "key_value_metadata="; (__isset.key_value_metadata ? (out << to_string(key_value_metadata)) : (out << "")); - out << ", " << "created_by="; (__isset.created_by ? (out << to_string(created_by)) : (out << "")); - out << ", " << "column_orders="; (__isset.column_orders ? (out << to_string(column_orders)) : (out << "")); - out << ")"; -} - - AesGcmV1::~AesGcmV1() throw() { } -void AesGcmV1::__set_aad_metadata(const std::string& val) { - this->aad_metadata = val; -__isset.aad_metadata = true; +void AesGcmV1::__set_aad_prefix(const std::string& val) { + this->aad_prefix = val; +__isset.aad_prefix = true; } -void AesGcmV1::__set_iv_prefix(const std::string& val) { - this->iv_prefix = val; -__isset.iv_prefix = true; +void AesGcmV1::__set_aad_file_unique(const std::string& val) { + this->aad_file_unique = val; +__isset.aad_file_unique = true; +} + +void AesGcmV1::__set_supply_aad_prefix(const bool val) { + this->supply_aad_prefix = val; +__isset.supply_aad_prefix = true; } std::ostream& operator<<(std::ostream& out, const AesGcmV1& obj) { @@ -6009,16 +5860,24 @@ uint32_t AesGcmV1::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->aad_metadata); - this->__isset.aad_metadata = true; + xfer += iprot->readBinary(this->aad_prefix); + this->__isset.aad_prefix = true; } else { xfer += iprot->skip(ftype); } break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->iv_prefix); - this->__isset.iv_prefix = true; + xfer += iprot->readBinary(this->aad_file_unique); + this->__isset.aad_file_unique = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->supply_aad_prefix); + this->__isset.supply_aad_prefix = true; } else { xfer += iprot->skip(ftype); } @@ -6040,14 +5899,19 @@ uint32_t AesGcmV1::write(::apache::thrift::protocol::TProtocol* oprot) const { ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); xfer += oprot->writeStructBegin("AesGcmV1"); - if (this->__isset.aad_metadata) { - xfer += oprot->writeFieldBegin("aad_metadata", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->aad_metadata); + if (this->__isset.aad_prefix) { + xfer += oprot->writeFieldBegin("aad_prefix", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeBinary(this->aad_prefix); xfer += oprot->writeFieldEnd(); } - if (this->__isset.iv_prefix) { - xfer += oprot->writeFieldBegin("iv_prefix", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->iv_prefix); + if (this->__isset.aad_file_unique) { + xfer += oprot->writeFieldBegin("aad_file_unique", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->aad_file_unique); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.supply_aad_prefix) { + xfer += oprot->writeFieldBegin("supply_aad_prefix", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->supply_aad_prefix); xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); @@ -6057,27 +5921,31 @@ uint32_t AesGcmV1::write(::apache::thrift::protocol::TProtocol* oprot) const { void swap(AesGcmV1 &a, AesGcmV1 &b) { using ::std::swap; - swap(a.aad_metadata, b.aad_metadata); - swap(a.iv_prefix, b.iv_prefix); + swap(a.aad_prefix, b.aad_prefix); + swap(a.aad_file_unique, b.aad_file_unique); + swap(a.supply_aad_prefix, b.supply_aad_prefix); swap(a.__isset, b.__isset); } -AesGcmV1::AesGcmV1(const AesGcmV1& other189) { - aad_metadata = other189.aad_metadata; - iv_prefix = other189.iv_prefix; - __isset = other189.__isset; +AesGcmV1::AesGcmV1(const AesGcmV1& other165) { + aad_prefix = other165.aad_prefix; + aad_file_unique = other165.aad_file_unique; + supply_aad_prefix = other165.supply_aad_prefix; + __isset = other165.__isset; } -AesGcmV1& AesGcmV1::operator=(const AesGcmV1& other190) { - aad_metadata = other190.aad_metadata; - iv_prefix = other190.iv_prefix; - __isset = other190.__isset; +AesGcmV1& AesGcmV1::operator=(const AesGcmV1& other166) { + aad_prefix = other166.aad_prefix; + aad_file_unique = other166.aad_file_unique; + supply_aad_prefix = other166.supply_aad_prefix; + __isset = other166.__isset; return *this; } void AesGcmV1::printTo(std::ostream& out) const { using ::apache::thrift::to_string; out << "AesGcmV1("; - out << "aad_metadata="; (__isset.aad_metadata ? (out << to_string(aad_metadata)) : (out << "")); - out << ", " << "iv_prefix="; (__isset.iv_prefix ? (out << to_string(iv_prefix)) : (out << "")); + out << "aad_prefix="; (__isset.aad_prefix ? (out << to_string(aad_prefix)) : (out << "")); + out << ", " << "aad_file_unique="; (__isset.aad_file_unique ? (out << to_string(aad_file_unique)) : (out << "")); + out << ", " << "supply_aad_prefix="; (__isset.supply_aad_prefix ? (out << to_string(supply_aad_prefix)) : (out << "")); out << ")"; } @@ -6086,19 +5954,19 @@ AesGcmCtrV1::~AesGcmCtrV1() throw() { } -void AesGcmCtrV1::__set_aad_metadata(const std::string& val) { - this->aad_metadata = val; -__isset.aad_metadata = true; +void AesGcmCtrV1::__set_aad_prefix(const std::string& val) { + this->aad_prefix = val; +__isset.aad_prefix = true; } -void AesGcmCtrV1::__set_gcm_iv_prefix(const std::string& val) { - this->gcm_iv_prefix = val; -__isset.gcm_iv_prefix = true; +void AesGcmCtrV1::__set_aad_file_unique(const std::string& val) { + this->aad_file_unique = val; +__isset.aad_file_unique = true; } -void AesGcmCtrV1::__set_ctr_iv_prefix(const std::string& val) { - this->ctr_iv_prefix = val; -__isset.ctr_iv_prefix = true; +void AesGcmCtrV1::__set_supply_aad_prefix(const bool val) { + this->supply_aad_prefix = val; +__isset.supply_aad_prefix = true; } std::ostream& operator<<(std::ostream& out, const AesGcmCtrV1& obj) { @@ -6130,24 +5998,24 @@ uint32_t AesGcmCtrV1::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->aad_metadata); - this->__isset.aad_metadata = true; + xfer += iprot->readBinary(this->aad_prefix); + this->__isset.aad_prefix = true; } else { xfer += iprot->skip(ftype); } break; case 2: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->gcm_iv_prefix); - this->__isset.gcm_iv_prefix = true; + xfer += iprot->readBinary(this->aad_file_unique); + this->__isset.aad_file_unique = true; } else { xfer += iprot->skip(ftype); } break; case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->ctr_iv_prefix); - this->__isset.ctr_iv_prefix = true; + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->supply_aad_prefix); + this->__isset.supply_aad_prefix = true; } else { xfer += iprot->skip(ftype); } @@ -6169,19 +6037,19 @@ uint32_t AesGcmCtrV1::write(::apache::thrift::protocol::TProtocol* oprot) const ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); xfer += oprot->writeStructBegin("AesGcmCtrV1"); - if (this->__isset.aad_metadata) { - xfer += oprot->writeFieldBegin("aad_metadata", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->aad_metadata); + if (this->__isset.aad_prefix) { + xfer += oprot->writeFieldBegin("aad_prefix", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeBinary(this->aad_prefix); xfer += oprot->writeFieldEnd(); } - if (this->__isset.gcm_iv_prefix) { - xfer += oprot->writeFieldBegin("gcm_iv_prefix", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->gcm_iv_prefix); + if (this->__isset.aad_file_unique) { + xfer += oprot->writeFieldBegin("aad_file_unique", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->aad_file_unique); xfer += oprot->writeFieldEnd(); } - if (this->__isset.ctr_iv_prefix) { - xfer += oprot->writeFieldBegin("ctr_iv_prefix", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeBinary(this->ctr_iv_prefix); + if (this->__isset.supply_aad_prefix) { + xfer += oprot->writeFieldBegin("supply_aad_prefix", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->supply_aad_prefix); xfer += oprot->writeFieldEnd(); } xfer += oprot->writeFieldStop(); @@ -6191,31 +6059,31 @@ uint32_t AesGcmCtrV1::write(::apache::thrift::protocol::TProtocol* oprot) const void swap(AesGcmCtrV1 &a, AesGcmCtrV1 &b) { using ::std::swap; - swap(a.aad_metadata, b.aad_metadata); - swap(a.gcm_iv_prefix, b.gcm_iv_prefix); - swap(a.ctr_iv_prefix, b.ctr_iv_prefix); + swap(a.aad_prefix, b.aad_prefix); + swap(a.aad_file_unique, b.aad_file_unique); + swap(a.supply_aad_prefix, b.supply_aad_prefix); swap(a.__isset, b.__isset); } -AesGcmCtrV1::AesGcmCtrV1(const AesGcmCtrV1& other191) { - aad_metadata = other191.aad_metadata; - gcm_iv_prefix = other191.gcm_iv_prefix; - ctr_iv_prefix = other191.ctr_iv_prefix; - __isset = other191.__isset; +AesGcmCtrV1::AesGcmCtrV1(const AesGcmCtrV1& other167) { + aad_prefix = other167.aad_prefix; + aad_file_unique = other167.aad_file_unique; + supply_aad_prefix = other167.supply_aad_prefix; + __isset = other167.__isset; } -AesGcmCtrV1& AesGcmCtrV1::operator=(const AesGcmCtrV1& other192) { - aad_metadata = other192.aad_metadata; - gcm_iv_prefix = other192.gcm_iv_prefix; - ctr_iv_prefix = other192.ctr_iv_prefix; - __isset = other192.__isset; +AesGcmCtrV1& AesGcmCtrV1::operator=(const AesGcmCtrV1& other168) { + aad_prefix = other168.aad_prefix; + aad_file_unique = other168.aad_file_unique; + supply_aad_prefix = other168.supply_aad_prefix; + __isset = other168.__isset; return *this; } void AesGcmCtrV1::printTo(std::ostream& out) const { using ::apache::thrift::to_string; out << "AesGcmCtrV1("; - out << "aad_metadata="; (__isset.aad_metadata ? (out << to_string(aad_metadata)) : (out << "")); - out << ", " << "gcm_iv_prefix="; (__isset.gcm_iv_prefix ? (out << to_string(gcm_iv_prefix)) : (out << "")); - out << ", " << "ctr_iv_prefix="; (__isset.ctr_iv_prefix ? (out << to_string(ctr_iv_prefix)) : (out << "")); + out << "aad_prefix="; (__isset.aad_prefix ? (out << to_string(aad_prefix)) : (out << "")); + out << ", " << "aad_file_unique="; (__isset.aad_file_unique ? (out << to_string(aad_file_unique)) : (out << "")); + out << ", " << "supply_aad_prefix="; (__isset.supply_aad_prefix ? (out << to_string(supply_aad_prefix)) : (out << "")); out << ")"; } @@ -6316,15 +6184,15 @@ void swap(EncryptionAlgorithm &a, EncryptionAlgorithm &b) { swap(a.__isset, b.__isset); } -EncryptionAlgorithm::EncryptionAlgorithm(const EncryptionAlgorithm& other193) { - AES_GCM_V1 = other193.AES_GCM_V1; - AES_GCM_CTR_V1 = other193.AES_GCM_CTR_V1; - __isset = other193.__isset; +EncryptionAlgorithm::EncryptionAlgorithm(const EncryptionAlgorithm& other169) { + AES_GCM_V1 = other169.AES_GCM_V1; + AES_GCM_CTR_V1 = other169.AES_GCM_CTR_V1; + __isset = other169.__isset; } -EncryptionAlgorithm& EncryptionAlgorithm::operator=(const EncryptionAlgorithm& other194) { - AES_GCM_V1 = other194.AES_GCM_V1; - AES_GCM_CTR_V1 = other194.AES_GCM_CTR_V1; - __isset = other194.__isset; +EncryptionAlgorithm& EncryptionAlgorithm::operator=(const EncryptionAlgorithm& other170) { + AES_GCM_V1 = other170.AES_GCM_V1; + AES_GCM_CTR_V1 = other170.AES_GCM_CTR_V1; + __isset = other170.__isset; return *this; } void EncryptionAlgorithm::printTo(std::ostream& out) const { @@ -6336,6 +6204,360 @@ void EncryptionAlgorithm::printTo(std::ostream& out) const { } +FileMetaData::~FileMetaData() throw() { +} + + +void FileMetaData::__set_version(const int32_t val) { + this->version = val; +} + +void FileMetaData::__set_schema(const std::vector & val) { + this->schema = val; +} + +void FileMetaData::__set_num_rows(const int64_t val) { + this->num_rows = val; +} + +void FileMetaData::__set_row_groups(const std::vector & val) { + this->row_groups = val; +} + +void FileMetaData::__set_key_value_metadata(const std::vector & val) { + this->key_value_metadata = val; +__isset.key_value_metadata = true; +} + +void FileMetaData::__set_created_by(const std::string& val) { + this->created_by = val; +__isset.created_by = true; +} + +void FileMetaData::__set_column_orders(const std::vector & val) { + this->column_orders = val; +__isset.column_orders = true; +} + +void FileMetaData::__set_encryption_algorithm(const EncryptionAlgorithm& val) { + this->encryption_algorithm = val; +__isset.encryption_algorithm = true; +} + +void FileMetaData::__set_footer_signing_key_metadata(const std::string& val) { + this->footer_signing_key_metadata = val; +__isset.footer_signing_key_metadata = true; +} +std::ostream& operator<<(std::ostream& out, const FileMetaData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t FileMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { + + ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + bool isset_version = false; + bool isset_schema = false; + bool isset_num_rows = false; + bool isset_row_groups = false; + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->version); + isset_version = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->schema.clear(); + uint32_t _size171; + ::apache::thrift::protocol::TType _etype174; + xfer += iprot->readListBegin(_etype174, _size171); + this->schema.resize(_size171); + uint32_t _i175; + for (_i175 = 0; _i175 < _size171; ++_i175) + { + xfer += this->schema[_i175].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_schema = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->num_rows); + isset_num_rows = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->row_groups.clear(); + uint32_t _size176; + ::apache::thrift::protocol::TType _etype179; + xfer += iprot->readListBegin(_etype179, _size176); + this->row_groups.resize(_size176); + uint32_t _i180; + for (_i180 = 0; _i180 < _size176; ++_i180) + { + xfer += this->row_groups[_i180].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_row_groups = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->key_value_metadata.clear(); + uint32_t _size181; + ::apache::thrift::protocol::TType _etype184; + xfer += iprot->readListBegin(_etype184, _size181); + this->key_value_metadata.resize(_size181); + uint32_t _i185; + for (_i185 = 0; _i185 < _size181; ++_i185) + { + xfer += this->key_value_metadata[_i185].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.key_value_metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->created_by); + this->__isset.created_by = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->column_orders.clear(); + uint32_t _size186; + ::apache::thrift::protocol::TType _etype189; + xfer += iprot->readListBegin(_etype189, _size186); + this->column_orders.resize(_size186); + uint32_t _i190; + for (_i190 = 0; _i190 < _size186; ++_i190) + { + xfer += this->column_orders[_i190].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.column_orders = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->encryption_algorithm.read(iprot); + this->__isset.encryption_algorithm = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->footer_signing_key_metadata); + this->__isset.footer_signing_key_metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_version) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_schema) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_num_rows) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_row_groups) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t FileMetaData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("FileMetaData"); + + xfer += oprot->writeFieldBegin("version", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32(this->version); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("schema", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->schema.size())); + std::vector ::const_iterator _iter191; + for (_iter191 = this->schema.begin(); _iter191 != this->schema.end(); ++_iter191) + { + xfer += (*_iter191).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("num_rows", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->num_rows); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("row_groups", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->row_groups.size())); + std::vector ::const_iterator _iter192; + for (_iter192 = this->row_groups.begin(); _iter192 != this->row_groups.end(); ++_iter192) + { + xfer += (*_iter192).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.key_value_metadata) { + xfer += oprot->writeFieldBegin("key_value_metadata", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->key_value_metadata.size())); + std::vector ::const_iterator _iter193; + for (_iter193 = this->key_value_metadata.begin(); _iter193 != this->key_value_metadata.end(); ++_iter193) + { + xfer += (*_iter193).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.created_by) { + xfer += oprot->writeFieldBegin("created_by", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->created_by); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.column_orders) { + xfer += oprot->writeFieldBegin("column_orders", ::apache::thrift::protocol::T_LIST, 7); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->column_orders.size())); + std::vector ::const_iterator _iter194; + for (_iter194 = this->column_orders.begin(); _iter194 != this->column_orders.end(); ++_iter194) + { + xfer += (*_iter194).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.encryption_algorithm) { + xfer += oprot->writeFieldBegin("encryption_algorithm", ::apache::thrift::protocol::T_STRUCT, 8); + xfer += this->encryption_algorithm.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.footer_signing_key_metadata) { + xfer += oprot->writeFieldBegin("footer_signing_key_metadata", ::apache::thrift::protocol::T_STRING, 9); + xfer += oprot->writeBinary(this->footer_signing_key_metadata); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(FileMetaData &a, FileMetaData &b) { + using ::std::swap; + swap(a.version, b.version); + swap(a.schema, b.schema); + swap(a.num_rows, b.num_rows); + swap(a.row_groups, b.row_groups); + swap(a.key_value_metadata, b.key_value_metadata); + swap(a.created_by, b.created_by); + swap(a.column_orders, b.column_orders); + swap(a.encryption_algorithm, b.encryption_algorithm); + swap(a.footer_signing_key_metadata, b.footer_signing_key_metadata); + swap(a.__isset, b.__isset); +} + +FileMetaData::FileMetaData(const FileMetaData& other195) { + version = other195.version; + schema = other195.schema; + num_rows = other195.num_rows; + row_groups = other195.row_groups; + key_value_metadata = other195.key_value_metadata; + created_by = other195.created_by; + column_orders = other195.column_orders; + encryption_algorithm = other195.encryption_algorithm; + footer_signing_key_metadata = other195.footer_signing_key_metadata; + __isset = other195.__isset; +} +FileMetaData& FileMetaData::operator=(const FileMetaData& other196) { + version = other196.version; + schema = other196.schema; + num_rows = other196.num_rows; + row_groups = other196.row_groups; + key_value_metadata = other196.key_value_metadata; + created_by = other196.created_by; + column_orders = other196.column_orders; + encryption_algorithm = other196.encryption_algorithm; + footer_signing_key_metadata = other196.footer_signing_key_metadata; + __isset = other196.__isset; + return *this; +} +void FileMetaData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "FileMetaData("; + out << "version=" << to_string(version); + out << ", " << "schema=" << to_string(schema); + out << ", " << "num_rows=" << to_string(num_rows); + out << ", " << "row_groups=" << to_string(row_groups); + out << ", " << "key_value_metadata="; (__isset.key_value_metadata ? (out << to_string(key_value_metadata)) : (out << "")); + out << ", " << "created_by="; (__isset.created_by ? (out << to_string(created_by)) : (out << "")); + out << ", " << "column_orders="; (__isset.column_orders ? (out << to_string(column_orders)) : (out << "")); + out << ", " << "encryption_algorithm="; (__isset.encryption_algorithm ? (out << to_string(encryption_algorithm)) : (out << "")); + out << ", " << "footer_signing_key_metadata="; (__isset.footer_signing_key_metadata ? (out << to_string(footer_signing_key_metadata)) : (out << "")); + out << ")"; +} + + FileCryptoMetaData::~FileCryptoMetaData() throw() { } @@ -6344,17 +6566,9 @@ void FileCryptoMetaData::__set_encryption_algorithm(const EncryptionAlgorithm& v this->encryption_algorithm = val; } -void FileCryptoMetaData::__set_encrypted_footer(const bool val) { - this->encrypted_footer = val; -} - -void FileCryptoMetaData::__set_footer_key_metadata(const std::string& val) { - this->footer_key_metadata = val; -__isset.footer_key_metadata = true; -} - -void FileCryptoMetaData::__set_footer_offset(const int64_t val) { - this->footer_offset = val; +void FileCryptoMetaData::__set_key_metadata(const std::string& val) { + this->key_metadata = val; +__isset.key_metadata = true; } std::ostream& operator<<(std::ostream& out, const FileCryptoMetaData& obj) { @@ -6376,8 +6590,6 @@ uint32_t FileCryptoMetaData::read(::apache::thrift::protocol::TProtocol* iprot) using ::apache::thrift::protocol::TProtocolException; bool isset_encryption_algorithm = false; - bool isset_encrypted_footer = false; - bool isset_footer_offset = false; while (true) { @@ -6396,25 +6608,9 @@ uint32_t FileCryptoMetaData::read(::apache::thrift::protocol::TProtocol* iprot) } break; case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->encrypted_footer); - isset_encrypted_footer = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->footer_key_metadata); - this->__isset.footer_key_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->footer_offset); - isset_footer_offset = true; + xfer += iprot->readBinary(this->key_metadata); + this->__isset.key_metadata = true; } else { xfer += iprot->skip(ftype); } @@ -6430,10 +6626,6 @@ uint32_t FileCryptoMetaData::read(::apache::thrift::protocol::TProtocol* iprot) if (!isset_encryption_algorithm) throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_encrypted_footer) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_footer_offset) - throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } @@ -6446,19 +6638,11 @@ uint32_t FileCryptoMetaData::write(::apache::thrift::protocol::TProtocol* oprot) xfer += this->encryption_algorithm.write(oprot); xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldBegin("encrypted_footer", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->encrypted_footer); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.footer_key_metadata) { - xfer += oprot->writeFieldBegin("footer_key_metadata", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeBinary(this->footer_key_metadata); + if (this->__isset.key_metadata) { + xfer += oprot->writeFieldBegin("key_metadata", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->key_metadata); xfer += oprot->writeFieldEnd(); } - xfer += oprot->writeFieldBegin("footer_offset", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->footer_offset); - xfer += oprot->writeFieldEnd(); - xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -6467,34 +6651,26 @@ uint32_t FileCryptoMetaData::write(::apache::thrift::protocol::TProtocol* oprot) void swap(FileCryptoMetaData &a, FileCryptoMetaData &b) { using ::std::swap; swap(a.encryption_algorithm, b.encryption_algorithm); - swap(a.encrypted_footer, b.encrypted_footer); - swap(a.footer_key_metadata, b.footer_key_metadata); - swap(a.footer_offset, b.footer_offset); + swap(a.key_metadata, b.key_metadata); swap(a.__isset, b.__isset); } -FileCryptoMetaData::FileCryptoMetaData(const FileCryptoMetaData& other195) { - encryption_algorithm = other195.encryption_algorithm; - encrypted_footer = other195.encrypted_footer; - footer_key_metadata = other195.footer_key_metadata; - footer_offset = other195.footer_offset; - __isset = other195.__isset; +FileCryptoMetaData::FileCryptoMetaData(const FileCryptoMetaData& other197) { + encryption_algorithm = other197.encryption_algorithm; + key_metadata = other197.key_metadata; + __isset = other197.__isset; } -FileCryptoMetaData& FileCryptoMetaData::operator=(const FileCryptoMetaData& other196) { - encryption_algorithm = other196.encryption_algorithm; - encrypted_footer = other196.encrypted_footer; - footer_key_metadata = other196.footer_key_metadata; - footer_offset = other196.footer_offset; - __isset = other196.__isset; +FileCryptoMetaData& FileCryptoMetaData::operator=(const FileCryptoMetaData& other198) { + encryption_algorithm = other198.encryption_algorithm; + key_metadata = other198.key_metadata; + __isset = other198.__isset; return *this; } void FileCryptoMetaData::printTo(std::ostream& out) const { using ::apache::thrift::to_string; out << "FileCryptoMetaData("; out << "encryption_algorithm=" << to_string(encryption_algorithm); - out << ", " << "encrypted_footer=" << to_string(encrypted_footer); - out << ", " << "footer_key_metadata="; (__isset.footer_key_metadata ? (out << to_string(footer_key_metadata)) : (out << "")); - out << ", " << "footer_offset=" << to_string(footer_offset); + out << ", " << "key_metadata="; (__isset.key_metadata ? (out << to_string(key_metadata)) : (out << "")); out << ")"; } diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h b/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h index a10e49be826..65e9e8226bb 100644 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h +++ b/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.11.0) + * Autogenerated by Thrift Compiler (0.12.0) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -17,7 +17,7 @@ #include -#include "parquet/util/windows_compatibility.h" +#include "parquet/windows_compatibility.h" namespace parquet { namespace format { @@ -161,6 +161,8 @@ class MilliSeconds; class MicroSeconds; +class NanoSeconds; + class TimeUnit; class TimestampType; @@ -215,14 +217,14 @@ class OffsetIndex; class ColumnIndex; -class FileMetaData; - class AesGcmV1; class AesGcmCtrV1; class EncryptionAlgorithm; +class FileMetaData; + class FileCryptoMetaData; typedef struct _Statistics__isset { @@ -629,10 +631,42 @@ void swap(MicroSeconds &a, MicroSeconds &b); std::ostream& operator<<(std::ostream& out, const MicroSeconds& obj); + +class NanoSeconds : public virtual ::apache::thrift::TBase { + public: + + NanoSeconds(const NanoSeconds&); + NanoSeconds& operator=(const NanoSeconds&); + NanoSeconds() { + } + + virtual ~NanoSeconds() throw(); + + bool operator == (const NanoSeconds & /* rhs */) const + { + return true; + } + bool operator != (const NanoSeconds &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NanoSeconds & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + + virtual void printTo(std::ostream& out) const; +}; + +void swap(NanoSeconds &a, NanoSeconds &b); + +std::ostream& operator<<(std::ostream& out, const NanoSeconds& obj); + typedef struct _TimeUnit__isset { - _TimeUnit__isset() : MILLIS(false), MICROS(false) {} + _TimeUnit__isset() : MILLIS(false), MICROS(false), NANOS(false) {} bool MILLIS :1; bool MICROS :1; + bool NANOS :1; } _TimeUnit__isset; class TimeUnit : public virtual ::apache::thrift::TBase { @@ -646,6 +680,7 @@ class TimeUnit : public virtual ::apache::thrift::TBase { virtual ~TimeUnit() throw(); MilliSeconds MILLIS; MicroSeconds MICROS; + NanoSeconds NANOS; _TimeUnit__isset __isset; @@ -653,6 +688,8 @@ class TimeUnit : public virtual ::apache::thrift::TBase { void __set_MICROS(const MicroSeconds& val); + void __set_NANOS(const NanoSeconds& val); + bool operator == (const TimeUnit & rhs) const { if (__isset.MILLIS != rhs.__isset.MILLIS) @@ -663,6 +700,10 @@ class TimeUnit : public virtual ::apache::thrift::TBase { return false; else if (__isset.MICROS && !(MICROS == rhs.MICROS)) return false; + if (__isset.NANOS != rhs.__isset.NANOS) + return false; + else if (__isset.NANOS && !(NANOS == rhs.NANOS)) + return false; return true; } bool operator != (const TimeUnit &rhs) const { @@ -867,7 +908,7 @@ void swap(BsonType &a, BsonType &b); std::ostream& operator<<(std::ostream& out, const BsonType& obj); typedef struct _LogicalType__isset { - _LogicalType__isset() : STRING(false), MAP(false), LIST(false), ENUM(false), DECIMAL(false), DATE(false), TIME(false), TIMESTAMP(false), INTEGER(false), UNKNOWN(false), JSON(false), BSON(false) {} + _LogicalType__isset() : STRING(false), MAP(false), LIST(false), ENUM(false), DECIMAL(false), DATE(false), TIME(false), TIMESTAMP(false), INTEGER(false), UNKNOWN(false), JSON(false), BSON(false), UUID(false) {} bool STRING :1; bool MAP :1; bool LIST :1; @@ -880,6 +921,7 @@ typedef struct _LogicalType__isset { bool UNKNOWN :1; bool JSON :1; bool BSON :1; + bool UUID :1; } _LogicalType__isset; class LogicalType : public virtual ::apache::thrift::TBase { @@ -903,6 +945,7 @@ class LogicalType : public virtual ::apache::thrift::TBase { NullType UNKNOWN; JsonType JSON; BsonType BSON; + UUIDType UUID; _LogicalType__isset __isset; @@ -930,6 +973,8 @@ class LogicalType : public virtual ::apache::thrift::TBase { void __set_BSON(const BsonType& val); + void __set_UUID(const UUIDType& val); + bool operator == (const LogicalType & rhs) const { if (__isset.STRING != rhs.__isset.STRING) @@ -980,6 +1025,10 @@ class LogicalType : public virtual ::apache::thrift::TBase { return false; else if (__isset.BSON && !(BSON == rhs.BSON)) return false; + if (__isset.UUID != rhs.__isset.UUID) + return false; + else if (__isset.UUID && !(UUID == rhs.UUID)) + return false; return true; } bool operator != (const LogicalType &rhs) const { @@ -1722,8 +1771,8 @@ void swap(EncryptionWithFooterKey &a, EncryptionWithFooterKey &b); std::ostream& operator<<(std::ostream& out, const EncryptionWithFooterKey& obj); typedef struct _EncryptionWithColumnKey__isset { - _EncryptionWithColumnKey__isset() : column_key_metadata(false) {} - bool column_key_metadata :1; + _EncryptionWithColumnKey__isset() : key_metadata(false) {} + bool key_metadata :1; } _EncryptionWithColumnKey__isset; class EncryptionWithColumnKey : public virtual ::apache::thrift::TBase { @@ -1731,26 +1780,26 @@ class EncryptionWithColumnKey : public virtual ::apache::thrift::TBase { EncryptionWithColumnKey(const EncryptionWithColumnKey&); EncryptionWithColumnKey& operator=(const EncryptionWithColumnKey&); - EncryptionWithColumnKey() : column_key_metadata() { + EncryptionWithColumnKey() : key_metadata() { } virtual ~EncryptionWithColumnKey() throw(); std::vector path_in_schema; - std::string column_key_metadata; + std::string key_metadata; _EncryptionWithColumnKey__isset __isset; void __set_path_in_schema(const std::vector & val); - void __set_column_key_metadata(const std::string& val); + void __set_key_metadata(const std::string& val); bool operator == (const EncryptionWithColumnKey & rhs) const { if (!(path_in_schema == rhs.path_in_schema)) return false; - if (__isset.column_key_metadata != rhs.__isset.column_key_metadata) + if (__isset.key_metadata != rhs.__isset.key_metadata) return false; - else if (__isset.column_key_metadata && !(column_key_metadata == rhs.column_key_metadata)) + else if (__isset.key_metadata && !(key_metadata == rhs.key_metadata)) return false; return true; } @@ -1823,14 +1872,15 @@ void swap(ColumnCryptoMetaData &a, ColumnCryptoMetaData &b); std::ostream& operator<<(std::ostream& out, const ColumnCryptoMetaData& obj); typedef struct _ColumnChunk__isset { - _ColumnChunk__isset() : file_path(false), meta_data(false), offset_index_offset(false), offset_index_length(false), column_index_offset(false), column_index_length(false), crypto_meta_data(false) {} + _ColumnChunk__isset() : file_path(false), meta_data(false), offset_index_offset(false), offset_index_length(false), column_index_offset(false), column_index_length(false), crypto_metadata(false), encrypted_column_metadata(false) {} bool file_path :1; bool meta_data :1; bool offset_index_offset :1; bool offset_index_length :1; bool column_index_offset :1; bool column_index_length :1; - bool crypto_meta_data :1; + bool crypto_metadata :1; + bool encrypted_column_metadata :1; } _ColumnChunk__isset; class ColumnChunk : public virtual ::apache::thrift::TBase { @@ -1838,7 +1888,7 @@ class ColumnChunk : public virtual ::apache::thrift::TBase { ColumnChunk(const ColumnChunk&); ColumnChunk& operator=(const ColumnChunk&); - ColumnChunk() : file_path(), file_offset(0), offset_index_offset(0), offset_index_length(0), column_index_offset(0), column_index_length(0) { + ColumnChunk() : file_path(), file_offset(0), offset_index_offset(0), offset_index_length(0), column_index_offset(0), column_index_length(0), encrypted_column_metadata() { } virtual ~ColumnChunk() throw(); @@ -1849,7 +1899,8 @@ class ColumnChunk : public virtual ::apache::thrift::TBase { int32_t offset_index_length; int64_t column_index_offset; int32_t column_index_length; - ColumnCryptoMetaData crypto_meta_data; + ColumnCryptoMetaData crypto_metadata; + std::string encrypted_column_metadata; _ColumnChunk__isset __isset; @@ -1867,7 +1918,9 @@ class ColumnChunk : public virtual ::apache::thrift::TBase { void __set_column_index_length(const int32_t val); - void __set_crypto_meta_data(const ColumnCryptoMetaData& val); + void __set_crypto_metadata(const ColumnCryptoMetaData& val); + + void __set_encrypted_column_metadata(const std::string& val); bool operator == (const ColumnChunk & rhs) const { @@ -1897,9 +1950,13 @@ class ColumnChunk : public virtual ::apache::thrift::TBase { return false; else if (__isset.column_index_length && !(column_index_length == rhs.column_index_length)) return false; - if (__isset.crypto_meta_data != rhs.__isset.crypto_meta_data) + if (__isset.crypto_metadata != rhs.__isset.crypto_metadata) return false; - else if (__isset.crypto_meta_data && !(crypto_meta_data == rhs.crypto_meta_data)) + else if (__isset.crypto_metadata && !(crypto_metadata == rhs.crypto_metadata)) + return false; + if (__isset.encrypted_column_metadata != rhs.__isset.encrypted_column_metadata) + return false; + else if (__isset.encrypted_column_metadata && !(encrypted_column_metadata == rhs.encrypted_column_metadata)) return false; return true; } @@ -1920,10 +1977,11 @@ void swap(ColumnChunk &a, ColumnChunk &b); std::ostream& operator<<(std::ostream& out, const ColumnChunk& obj); typedef struct _RowGroup__isset { - _RowGroup__isset() : sorting_columns(false), file_offset(false), total_compressed_size(false) {} + _RowGroup__isset() : sorting_columns(false), file_offset(false), total_compressed_size(false), ordinal(false) {} bool sorting_columns :1; bool file_offset :1; bool total_compressed_size :1; + bool ordinal :1; } _RowGroup__isset; class RowGroup : public virtual ::apache::thrift::TBase { @@ -1931,7 +1989,7 @@ class RowGroup : public virtual ::apache::thrift::TBase { RowGroup(const RowGroup&); RowGroup& operator=(const RowGroup&); - RowGroup() : total_byte_size(0), num_rows(0), file_offset(0), total_compressed_size(0) { + RowGroup() : total_byte_size(0), num_rows(0), file_offset(0), total_compressed_size(0), ordinal(0) { } virtual ~RowGroup() throw(); @@ -1941,6 +1999,7 @@ class RowGroup : public virtual ::apache::thrift::TBase { std::vector sorting_columns; int64_t file_offset; int64_t total_compressed_size; + int16_t ordinal; _RowGroup__isset __isset; @@ -1956,6 +2015,8 @@ class RowGroup : public virtual ::apache::thrift::TBase { void __set_total_compressed_size(const int64_t val); + void __set_ordinal(const int16_t val); + bool operator == (const RowGroup & rhs) const { if (!(columns == rhs.columns)) @@ -1976,6 +2037,10 @@ class RowGroup : public virtual ::apache::thrift::TBase { return false; else if (__isset.total_compressed_size && !(total_compressed_size == rhs.total_compressed_size)) return false; + if (__isset.ordinal != rhs.__isset.ordinal) + return false; + else if (__isset.ordinal && !(ordinal == rhs.ordinal)) + return false; return true; } bool operator != (const RowGroup &rhs) const { @@ -2215,90 +2280,11 @@ void swap(ColumnIndex &a, ColumnIndex &b); std::ostream& operator<<(std::ostream& out, const ColumnIndex& obj); -typedef struct _FileMetaData__isset { - _FileMetaData__isset() : key_value_metadata(false), created_by(false), column_orders(false) {} - bool key_value_metadata :1; - bool created_by :1; - bool column_orders :1; -} _FileMetaData__isset; - -class FileMetaData : public virtual ::apache::thrift::TBase { - public: - - FileMetaData(const FileMetaData&); - FileMetaData& operator=(const FileMetaData&); - FileMetaData() : version(0), num_rows(0), created_by() { - } - - virtual ~FileMetaData() throw(); - int32_t version; - std::vector schema; - int64_t num_rows; - std::vector row_groups; - std::vector key_value_metadata; - std::string created_by; - std::vector column_orders; - - _FileMetaData__isset __isset; - - void __set_version(const int32_t val); - - void __set_schema(const std::vector & val); - - void __set_num_rows(const int64_t val); - - void __set_row_groups(const std::vector & val); - - void __set_key_value_metadata(const std::vector & val); - - void __set_created_by(const std::string& val); - - void __set_column_orders(const std::vector & val); - - bool operator == (const FileMetaData & rhs) const - { - if (!(version == rhs.version)) - return false; - if (!(schema == rhs.schema)) - return false; - if (!(num_rows == rhs.num_rows)) - return false; - if (!(row_groups == rhs.row_groups)) - return false; - if (__isset.key_value_metadata != rhs.__isset.key_value_metadata) - return false; - else if (__isset.key_value_metadata && !(key_value_metadata == rhs.key_value_metadata)) - return false; - if (__isset.created_by != rhs.__isset.created_by) - return false; - else if (__isset.created_by && !(created_by == rhs.created_by)) - return false; - if (__isset.column_orders != rhs.__isset.column_orders) - return false; - else if (__isset.column_orders && !(column_orders == rhs.column_orders)) - return false; - return true; - } - bool operator != (const FileMetaData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FileMetaData & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(FileMetaData &a, FileMetaData &b); - -std::ostream& operator<<(std::ostream& out, const FileMetaData& obj); - typedef struct _AesGcmV1__isset { - _AesGcmV1__isset() : aad_metadata(false), iv_prefix(false) {} - bool aad_metadata :1; - bool iv_prefix :1; + _AesGcmV1__isset() : aad_prefix(false), aad_file_unique(false), supply_aad_prefix(false) {} + bool aad_prefix :1; + bool aad_file_unique :1; + bool supply_aad_prefix :1; } _AesGcmV1__isset; class AesGcmV1 : public virtual ::apache::thrift::TBase { @@ -2306,28 +2292,35 @@ class AesGcmV1 : public virtual ::apache::thrift::TBase { AesGcmV1(const AesGcmV1&); AesGcmV1& operator=(const AesGcmV1&); - AesGcmV1() : aad_metadata(), iv_prefix() { + AesGcmV1() : aad_prefix(), aad_file_unique(), supply_aad_prefix(0) { } virtual ~AesGcmV1() throw(); - std::string aad_metadata; - std::string iv_prefix; + std::string aad_prefix; + std::string aad_file_unique; + bool supply_aad_prefix; _AesGcmV1__isset __isset; - void __set_aad_metadata(const std::string& val); + void __set_aad_prefix(const std::string& val); - void __set_iv_prefix(const std::string& val); + void __set_aad_file_unique(const std::string& val); + + void __set_supply_aad_prefix(const bool val); bool operator == (const AesGcmV1 & rhs) const { - if (__isset.aad_metadata != rhs.__isset.aad_metadata) + if (__isset.aad_prefix != rhs.__isset.aad_prefix) return false; - else if (__isset.aad_metadata && !(aad_metadata == rhs.aad_metadata)) + else if (__isset.aad_prefix && !(aad_prefix == rhs.aad_prefix)) return false; - if (__isset.iv_prefix != rhs.__isset.iv_prefix) + if (__isset.aad_file_unique != rhs.__isset.aad_file_unique) return false; - else if (__isset.iv_prefix && !(iv_prefix == rhs.iv_prefix)) + else if (__isset.aad_file_unique && !(aad_file_unique == rhs.aad_file_unique)) + return false; + if (__isset.supply_aad_prefix != rhs.__isset.supply_aad_prefix) + return false; + else if (__isset.supply_aad_prefix && !(supply_aad_prefix == rhs.supply_aad_prefix)) return false; return true; } @@ -2348,10 +2341,10 @@ void swap(AesGcmV1 &a, AesGcmV1 &b); std::ostream& operator<<(std::ostream& out, const AesGcmV1& obj); typedef struct _AesGcmCtrV1__isset { - _AesGcmCtrV1__isset() : aad_metadata(false), gcm_iv_prefix(false), ctr_iv_prefix(false) {} - bool aad_metadata :1; - bool gcm_iv_prefix :1; - bool ctr_iv_prefix :1; + _AesGcmCtrV1__isset() : aad_prefix(false), aad_file_unique(false), supply_aad_prefix(false) {} + bool aad_prefix :1; + bool aad_file_unique :1; + bool supply_aad_prefix :1; } _AesGcmCtrV1__isset; class AesGcmCtrV1 : public virtual ::apache::thrift::TBase { @@ -2359,35 +2352,35 @@ class AesGcmCtrV1 : public virtual ::apache::thrift::TBase { AesGcmCtrV1(const AesGcmCtrV1&); AesGcmCtrV1& operator=(const AesGcmCtrV1&); - AesGcmCtrV1() : aad_metadata(), gcm_iv_prefix(), ctr_iv_prefix() { + AesGcmCtrV1() : aad_prefix(), aad_file_unique(), supply_aad_prefix(0) { } virtual ~AesGcmCtrV1() throw(); - std::string aad_metadata; - std::string gcm_iv_prefix; - std::string ctr_iv_prefix; + std::string aad_prefix; + std::string aad_file_unique; + bool supply_aad_prefix; _AesGcmCtrV1__isset __isset; - void __set_aad_metadata(const std::string& val); + void __set_aad_prefix(const std::string& val); - void __set_gcm_iv_prefix(const std::string& val); + void __set_aad_file_unique(const std::string& val); - void __set_ctr_iv_prefix(const std::string& val); + void __set_supply_aad_prefix(const bool val); bool operator == (const AesGcmCtrV1 & rhs) const { - if (__isset.aad_metadata != rhs.__isset.aad_metadata) + if (__isset.aad_prefix != rhs.__isset.aad_prefix) return false; - else if (__isset.aad_metadata && !(aad_metadata == rhs.aad_metadata)) + else if (__isset.aad_prefix && !(aad_prefix == rhs.aad_prefix)) return false; - if (__isset.gcm_iv_prefix != rhs.__isset.gcm_iv_prefix) + if (__isset.aad_file_unique != rhs.__isset.aad_file_unique) return false; - else if (__isset.gcm_iv_prefix && !(gcm_iv_prefix == rhs.gcm_iv_prefix)) + else if (__isset.aad_file_unique && !(aad_file_unique == rhs.aad_file_unique)) return false; - if (__isset.ctr_iv_prefix != rhs.__isset.ctr_iv_prefix) + if (__isset.supply_aad_prefix != rhs.__isset.supply_aad_prefix) return false; - else if (__isset.ctr_iv_prefix && !(ctr_iv_prefix == rhs.ctr_iv_prefix)) + else if (__isset.supply_aad_prefix && !(supply_aad_prefix == rhs.supply_aad_prefix)) return false; return true; } @@ -2459,9 +2452,105 @@ void swap(EncryptionAlgorithm &a, EncryptionAlgorithm &b); std::ostream& operator<<(std::ostream& out, const EncryptionAlgorithm& obj); +typedef struct _FileMetaData__isset { + _FileMetaData__isset() : key_value_metadata(false), created_by(false), column_orders(false), encryption_algorithm(false), footer_signing_key_metadata(false) {} + bool key_value_metadata :1; + bool created_by :1; + bool column_orders :1; + bool encryption_algorithm :1; + bool footer_signing_key_metadata :1; +} _FileMetaData__isset; + +class FileMetaData : public virtual ::apache::thrift::TBase { + public: + + FileMetaData(const FileMetaData&); + FileMetaData& operator=(const FileMetaData&); + FileMetaData() : version(0), num_rows(0), created_by(), footer_signing_key_metadata() { + } + + virtual ~FileMetaData() throw(); + int32_t version; + std::vector schema; + int64_t num_rows; + std::vector row_groups; + std::vector key_value_metadata; + std::string created_by; + std::vector column_orders; + EncryptionAlgorithm encryption_algorithm; + std::string footer_signing_key_metadata; + + _FileMetaData__isset __isset; + + void __set_version(const int32_t val); + + void __set_schema(const std::vector & val); + + void __set_num_rows(const int64_t val); + + void __set_row_groups(const std::vector & val); + + void __set_key_value_metadata(const std::vector & val); + + void __set_created_by(const std::string& val); + + void __set_column_orders(const std::vector & val); + + void __set_encryption_algorithm(const EncryptionAlgorithm& val); + + void __set_footer_signing_key_metadata(const std::string& val); + + bool operator == (const FileMetaData & rhs) const + { + if (!(version == rhs.version)) + return false; + if (!(schema == rhs.schema)) + return false; + if (!(num_rows == rhs.num_rows)) + return false; + if (!(row_groups == rhs.row_groups)) + return false; + if (__isset.key_value_metadata != rhs.__isset.key_value_metadata) + return false; + else if (__isset.key_value_metadata && !(key_value_metadata == rhs.key_value_metadata)) + return false; + if (__isset.created_by != rhs.__isset.created_by) + return false; + else if (__isset.created_by && !(created_by == rhs.created_by)) + return false; + if (__isset.column_orders != rhs.__isset.column_orders) + return false; + else if (__isset.column_orders && !(column_orders == rhs.column_orders)) + return false; + if (__isset.encryption_algorithm != rhs.__isset.encryption_algorithm) + return false; + else if (__isset.encryption_algorithm && !(encryption_algorithm == rhs.encryption_algorithm)) + return false; + if (__isset.footer_signing_key_metadata != rhs.__isset.footer_signing_key_metadata) + return false; + else if (__isset.footer_signing_key_metadata && !(footer_signing_key_metadata == rhs.footer_signing_key_metadata)) + return false; + return true; + } + bool operator != (const FileMetaData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const FileMetaData & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + + virtual void printTo(std::ostream& out) const; +}; + +void swap(FileMetaData &a, FileMetaData &b); + +std::ostream& operator<<(std::ostream& out, const FileMetaData& obj); + typedef struct _FileCryptoMetaData__isset { - _FileCryptoMetaData__isset() : footer_key_metadata(false) {} - bool footer_key_metadata :1; + _FileCryptoMetaData__isset() : key_metadata(false) {} + bool key_metadata :1; } _FileCryptoMetaData__isset; class FileCryptoMetaData : public virtual ::apache::thrift::TBase { @@ -2469,36 +2558,26 @@ class FileCryptoMetaData : public virtual ::apache::thrift::TBase { FileCryptoMetaData(const FileCryptoMetaData&); FileCryptoMetaData& operator=(const FileCryptoMetaData&); - FileCryptoMetaData() : encrypted_footer(0), footer_key_metadata(), footer_offset(0) { + FileCryptoMetaData() : key_metadata() { } virtual ~FileCryptoMetaData() throw(); EncryptionAlgorithm encryption_algorithm; - bool encrypted_footer; - std::string footer_key_metadata; - int64_t footer_offset; + std::string key_metadata; _FileCryptoMetaData__isset __isset; void __set_encryption_algorithm(const EncryptionAlgorithm& val); - void __set_encrypted_footer(const bool val); - - void __set_footer_key_metadata(const std::string& val); - - void __set_footer_offset(const int64_t val); + void __set_key_metadata(const std::string& val); bool operator == (const FileCryptoMetaData & rhs) const { if (!(encryption_algorithm == rhs.encryption_algorithm)) return false; - if (!(encrypted_footer == rhs.encrypted_footer)) + if (__isset.key_metadata != rhs.__isset.key_metadata) return false; - if (__isset.footer_key_metadata != rhs.__isset.footer_key_metadata) - return false; - else if (__isset.footer_key_metadata && !(footer_key_metadata == rhs.footer_key_metadata)) - return false; - if (!(footer_offset == rhs.footer_offset)) + else if (__isset.key_metadata && !(key_metadata == rhs.key_metadata)) return false; return true; } diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index edb8d5c15f4..60a366d7171 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -61,14 +61,14 @@ namespace DB /// Inserts numeric data right into internal column data to reduce an overhead template > - static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = static_cast(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - std::shared_ptr chunk = arrow_column->data()->chunk(chunk_i); + std::shared_ptr chunk = arrow_column->chunk(chunk_i); /// buffers[0] is a null bitmap and buffers[1] are actual values std::shared_ptr buffer = chunk->data()->buffers[1]; @@ -80,15 +80,15 @@ namespace DB /// Inserts chars and offsets right into internal column data to reduce an overhead. /// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. /// Also internal strings are null terminated. - static void fillColumnWithStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); size_t chars_t_size = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - arrow::BinaryArray & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + arrow::BinaryArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); const size_t chunk_length = chunk.length(); chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1); @@ -98,9 +98,9 @@ namespace DB column_chars_t.reserve(chars_t_size); column_offsets.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - arrow::BinaryArray & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + arrow::BinaryArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); std::shared_ptr buffer = chunk.value_data(); const size_t chunk_length = chunk.length(); @@ -118,14 +118,14 @@ namespace DB } } - static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - arrow::BooleanArray & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + arrow::BooleanArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); /// buffers[0] is a null bitmap and buffers[1] are actual values std::shared_ptr buffer = chunk.data()->buffers[1]; @@ -135,14 +135,14 @@ namespace DB } /// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving - static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - arrow::Date32Array & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + arrow::Date32Array & chunk = static_cast(*(arrow_column->chunk(chunk_i))); for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { @@ -150,7 +150,7 @@ namespace DB if (days_num > DATE_LUT_MAX_DAY_NUM) { // TODO: will it rollback correctly? - throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + arrow_column->name() + throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column->getName() + "\" is greater than " "max allowed Date value, which is " + std::to_string(DATE_LUT_MAX_DAY_NUM), @@ -163,14 +163,14 @@ namespace DB } /// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving - static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms @@ -179,14 +179,14 @@ namespace DB } } - static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = assert_cast &>(*internal_column).getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); const auto & type = static_cast(*chunk.type()); UInt32 divide = 1; @@ -215,15 +215,15 @@ namespace DB } } - static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column = assert_cast &>(*internal_column); auto & column_data = column.getData(); column_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto & chunk = static_cast(*(arrow_column->data()->chunk(chunk_i))); + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { column_data.emplace_back(chunk.IsNull(value_i) ? Decimal128(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column @@ -232,14 +232,14 @@ namespace DB } /// Creates a null bytemap from arrow's null bitmap - static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) + static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) { PaddedPODArray & bytemap_data = assert_cast &>(*bytemap).getData(); bytemap_data.reserve(arrow_column->length()); - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->data()->num_chunks()); ++chunk_i) + for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) { - std::shared_ptr chunk = arrow_column->data()->chunk(chunk_i); + std::shared_ptr chunk = arrow_column->chunk(chunk_i); for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) bytemap_data.emplace_back(chunk->IsNull(value_i)); @@ -255,7 +255,7 @@ namespace DB columns_list.reserve(header.rows()); - using NameToColumnPtr = std::unordered_map>; + using NameToColumnPtr = std::unordered_map>; if (!read_status.ok()) throw Exception{"Error while reading " + format_name + " data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; @@ -270,10 +270,10 @@ namespace DB ++row_group_current; NameToColumnPtr name_to_column_ptr; - for (size_t i = 0, num_columns = static_cast(table->num_columns()); i < num_columns; ++i) + for (const auto column_name : table->ColumnNames()) { - std::shared_ptr arrow_column = table->column(i); - name_to_column_ptr[arrow_column->name()] = arrow_column; + std::shared_ptr arrow_column = table->GetColumnByName(column_name); + name_to_column_ptr[column_name] = arrow_column; } for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) @@ -285,7 +285,7 @@ namespace DB throw Exception{"Column \"" + header_column.name + "\" is not presented in input data", ErrorCodes::THERE_IS_NO_COLUMN}; - std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; + std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; arrow::Type::type arrow_type = arrow_column->type()->id(); // TODO: check if a column is const? @@ -313,7 +313,7 @@ namespace DB } else { - throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() + throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + header_column.name + "\" is not supported for conversion from a " + format_name + " data format", ErrorCodes::CANNOT_CONVERT_TYPE}; } @@ -374,7 +374,7 @@ namespace DB throw Exception { "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" - + arrow_column->name() + "\"", + + header_column.name + "\"", ErrorCodes::UNKNOWN_TYPE }; } diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 55c2ebccf41..1ca75fd40e3 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -44,9 +44,11 @@ namespace DB buffer = std::make_unique(file_data); // TODO: maybe use parquet::RandomAccessSource? - auto reader = parquet::ParquetFileReader::Open(std::make_shared<::arrow::io::BufferReader>(*buffer)); - file_reader = std::make_unique(::arrow::default_memory_pool(), - std::move(reader)); + auto status = parquet::arrow::FileReader::Make( + ::arrow::default_memory_pool(), + parquet::ParquetFileReader::Open(std::make_shared<::arrow::io::BufferReader>(*buffer)), + &file_reader); + row_group_total = file_reader->num_row_groups(); row_group_current = 0; } diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index f7092bc9a5b..e71bab88d9e 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -21,9 +21,10 @@ #include #include #include +#include #include #include -#include +#include namespace DB @@ -238,22 +239,39 @@ static const PaddedPODArray * extractNullBytemapPtr(ColumnPtr column) } -class OstreamOutputStream : public parquet::OutputStream +class OstreamOutputStream : public arrow::io::OutputStream { public: explicit OstreamOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {} - virtual ~OstreamOutputStream() {} - virtual void Close() {} - virtual int64_t Tell() { return total_length; } - virtual void Write(const uint8_t * data, int64_t length) - { + virtual ~OstreamOutputStream() { + is_open_ = true; + } + + // FileInterface + ::arrow::Status Close() override{ + is_open_ = false; + return ::arrow::Status::OK(); + } + ::arrow::Status Tell(int64_t* position) const override { + *position = total_length; + return ::arrow::Status::OK(); + }; + bool closed() const override{ + return is_open_; + }; + + // Writable + ::arrow::Status Write(const void* data, int64_t length) override { ostr.write(reinterpret_cast(data), length); total_length += length; + // TODO try catch write ? + return ::arrow::Status::OK(); } private: WriteBuffer & ostr; int64_t total_length = 0; + bool is_open_ = false; PARQUET_DISALLOW_COPY_AND_ASSIGN(OstreamOutputStream); }; @@ -396,7 +414,6 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) arrow::default_memory_pool(), sink, props, /*parquet::default_writer_properties(),*/ - parquet::arrow::default_arrow_writer_properties(), &file_writer); if (!status.ok()) throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; From 5d12b7e24b42694965a5f1d08da57ca32130acc0 Mon Sep 17 00:00:00 2001 From: Maxim Ulanovskiy Date: Sun, 22 Dec 2019 15:24:23 +0300 Subject: [PATCH 13/66] added submodule flatbuffers v1.11.0 --- .gitmodules | 3 +++ contrib/flatbuffers | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/flatbuffers diff --git a/.gitmodules b/.gitmodules index b399bf7b94d..b115df22b99 100644 --- a/.gitmodules +++ b/.gitmodules @@ -128,3 +128,6 @@ [submodule "contrib/icu"] path = contrib/icu url = https://github.com/unicode-org/icu.git +[submodule "contrib/flatbuffers"] + path = contrib/flatbuffers + url = git@github.com:google/flatbuffers.git diff --git a/contrib/flatbuffers b/contrib/flatbuffers new file mode 160000 index 00000000000..bf9eb67ab93 --- /dev/null +++ b/contrib/flatbuffers @@ -0,0 +1 @@ +Subproject commit bf9eb67ab9371755c6bcece13cadc7693bcbf264 From 5f8f5458f230d5f1ac410d67668d2256d9d105d8 Mon Sep 17 00:00:00 2001 From: dinosaur Date: Sun, 22 Dec 2019 21:53:37 +0800 Subject: [PATCH 14/66] fix_8325 --- .../src/AggregateFunctions/HelpersMinMaxAny.h | 33 ++++++++++++++----- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/dbms/src/AggregateFunctions/HelpersMinMaxAny.h b/dbms/src/AggregateFunctions/HelpersMinMaxAny.h index 8457e33dcbb..ec83ae4e981 100644 --- a/dbms/src/AggregateFunctions/HelpersMinMaxAny.h +++ b/dbms/src/AggregateFunctions/HelpersMinMaxAny.h @@ -52,26 +52,43 @@ template